You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2015/11/26 14:13:43 UTC

[01/43] ignite git commit: Make binary marshaller default WIP.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5-cache-get-opts 54ae69021 -> 1eaa8c92d


http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
index 79909c3..aa58054 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-grid1.xml
@@ -86,10 +86,5 @@
                 </property>
             </bean>
         </property>
-
-        <!-- Binary marshaller configuration -->
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.BinaryMarshaller"/>
-        </property>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
index 2f7d746..366ac13 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/marshaller-explicit.xml
@@ -32,7 +32,7 @@
         </property>
 
         <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.BinaryMarshaller"/>
+            <bean class="org.apache.ignite.internal.portable.BinaryMarshaller"/>
         </property>
 
         <property name="discoverySpi">


[43/43] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 1eaa8c92d2db395279fb33cdd8e914bd2f2f9f3d
Parents: 54ae690 c584cca
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Nov 26 15:12:49 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Nov 26 15:12:49 2015 +0300

----------------------------------------------------------------------
 .../config/binary/example-ignite-binary.xml     |  44 ----
 examples/config/example-ignite.xml              |  10 +-
 .../ignite/examples/binary/EmployeeKey.java     |   3 +
 .../binary/ExampleBinaryNodeStartup.java        |  36 ---
 ...ComputeClientBinaryTaskExecutionExample.java |   6 +-
 .../CacheClientBinaryPutGetExample.java         |   6 +-
 .../datagrid/CacheClientBinaryQueryExample.java |   6 +-
 .../scalar/examples/ScalarCacheExample.scala    |  19 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   1 +
 .../java/org/apache/ignite/IgniteBinary.java    |  15 +-
 .../java/org/apache/ignite/IgniteCache.java     |   3 +-
 .../apache/ignite/binary/BinaryIdMapper.java    |   3 +-
 .../org/apache/ignite/binary/BinaryObject.java  |   9 +-
 .../org/apache/ignite/binary/BinaryType.java    |   9 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |   2 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |   2 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   3 +-
 .../configuration/CacheConfiguration.java       |  36 ++-
 .../internal/GridEventConsumeHandler.java       |   5 +
 .../internal/GridMessageListenHandler.java      |   5 +
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   6 +-
 .../ignite/internal/NodeStoppingException.java  |  35 +++
 .../discovery/GridDiscoveryManager.java         |  19 +-
 .../internal/portable/BinaryFieldImpl.java      |   6 +-
 .../internal/portable/BinaryMarshaller.java     | 144 +++++++++++
 .../internal/portable/BinaryObjectEx.java       | 225 +----------------
 .../internal/portable/BinaryObjectExImpl.java   | 245 +++++++++++++++++++
 .../internal/portable/BinaryObjectImpl.java     |   2 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryTypeImpl.java       |   5 +
 .../portable/PortableClassDescriptor.java       |   2 +-
 .../internal/portable/PortableContext.java      |  35 ++-
 .../ignite/internal/portable/PortableUtils.java |   2 +-
 .../builder/BinaryObjectBuilderImpl.java        |   9 +-
 .../builder/PortableBuilderSerializer.java      |   4 +-
 .../processors/cache/CacheObjectContext.java    |   2 +-
 .../cache/GridCacheDeploymentManager.java       |   6 +-
 .../processors/cache/GridCacheEventManager.java |  38 ++-
 .../processors/cache/GridCacheIoManager.java    |   4 +
 .../processors/cache/GridCacheMessage.java      |   3 +-
 .../processors/cache/GridCacheMvccManager.java  |   3 +-
 .../processors/cache/GridCacheProcessor.java    |   4 +-
 .../processors/cache/IgniteCacheProxy.java      |  14 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |  19 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   1 +
 .../dht/GridPartitionedSingleGetFuture.java     |   2 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   2 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   3 +-
 .../CacheObjectBinaryProcessorImpl.java         |  33 ++-
 .../cache/portable/IgniteBinaryImpl.java        |   9 +-
 .../cache/query/GridCacheSqlQuery.java          |  42 ++++
 .../cache/query/GridCacheTwoStepQuery.java      |  35 ++-
 .../continuous/CacheContinuousQueryEntry.java   |  50 +++-
 .../continuous/CacheContinuousQueryEvent.java   |   6 +-
 .../continuous/CacheContinuousQueryHandler.java |  32 ++-
 .../CacheContinuousQueryListener.java           |   5 +
 .../continuous/CacheContinuousQueryManager.java |  57 +++--
 .../cache/store/CacheOsStoreManager.java        |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  29 ++-
 .../processors/cacheobject/NoOpBinary.java      |   8 +-
 .../continuous/GridContinuousHandler.java       |   5 +
 .../continuous/GridContinuousProcessor.java     |  10 +-
 .../StartRoutineDiscoveryMessage.java           |  13 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   2 +-
 .../processors/query/GridQueryProcessor.java    |  68 +++--
 .../processors/query/GridQueryProperty.java     |  45 ++++
 .../query/GridQueryTypeDescriptor.java          |   6 +
 .../visor/cache/VisorCacheConfiguration.java    |   4 +-
 .../cache/VisorCacheTypeFieldMetadata.java      |  30 ++-
 .../visor/cache/VisorCacheTypeMetadata.java     | 133 +++++++++-
 .../marshaller/portable/BinaryMarshaller.java   | 146 -----------
 .../META-INF/classnames-jdk.properties          |   2 +
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |  85 +++++++
 ...JdbcPojoStorePortableMarshallerSelfTest.java |  85 -------
 .../IgniteClientReconnectCacheTest.java         |   7 +
 .../portable/BinaryFieldsAbstractSelfTest.java  |  13 +-
 .../portable/BinaryFieldsHeapSelfTest.java      |   4 +-
 .../portable/BinaryFieldsOffheapSelfTest.java   |   3 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   5 +-
 .../BinaryFooterOffsetsHeapSelfTest.java        |   4 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   3 +-
 .../portable/BinaryMarshallerSelfTest.java      |  17 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   7 +-
 .../portable/BinaryObjectBuilderSelfTest.java   | 102 +++++---
 .../GridPortableAffinityKeySelfTest.java        |  13 +-
 ...idPortableMarshallerCtxDisabledSelfTest.java |   1 -
 .../portable/GridPortableMetaDataSelfTest.java  |  11 +-
 .../portable/GridPortableWildcardsSelfTest.java |   1 -
 .../cache/GridCacheAbstractFullApiSelfTest.java |  97 ++++++++
 ...IgniteCacheAbstractExecutionContextTest.java |   2 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |   3 +
 .../IgniteCollectionAbstractTest.java           |   3 +
 .../IgniteCacheSizeFailoverTest.java            |   7 +
 .../IgniteCachePutRetryAbstractSelfTest.java    |   4 +
 ...naryObjectsAbstractDataStreamerSelfTest.java |   2 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |   2 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |  10 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   6 +-
 .../GridCachePortableStoreAbstractSelfTest.java |   2 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   2 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |   2 +-
 .../DataStreamProcessorPortableSelfTest.java    |   2 +-
 .../GridDataStreamerImplSelfTest.java           |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 .../GridCacheMemoryModePortableSelfTest.java    |   2 +-
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   2 +-
 ...eapTieredEvictionAtomicPortableSelfTest.java |   2 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |   2 +-
 .../GridCacheOffHeapTieredPortableSelfTest.java |   2 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |   2 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   2 +-
 .../PlatformComputeBinarizableArgTask.java      |   8 +-
 .../ignite/testframework/junits/IgniteMock.java |  39 ++-
 .../IgniteBinaryObjectsCacheTestSuite2.java     |  33 +++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   2 +-
 .../IgnitePortableCacheTestSuite.java           |   2 +-
 .../query/h2/GridH2ResultSetIterator.java       |  19 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 227 ++++++++++++++---
 .../query/h2/opt/GridH2RowDescriptor.java       |   5 +
 .../processors/query/h2/opt/GridH2Table.java    |  69 +++++-
 .../query/h2/opt/GridH2TreeIndex.java           |  98 +++++---
 .../query/h2/sql/GridSqlQuerySplitter.java      |  73 ++++--
 .../processors/query/h2/sql/GridSqlSelect.java  |  27 ++
 .../query/h2/twostep/GridMapQueryExecutor.java  |  17 +-
 .../query/h2/twostep/GridMergeIndex.java        |   7 +
 .../h2/twostep/GridMergeIndexUnsorted.java      |  14 ++
 .../h2/twostep/GridReduceQueryExecutor.java     | 117 ++++++---
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  23 ++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  71 +++---
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   2 -
 ...PartitionedSnapshotEnabledQuerySelfTest.java |  28 +++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  19 ++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 .../IgnitePortableCacheQueryTestSuite.java      |   2 +-
 .../Config/Compute/compute-grid1.xml            |   5 -
 .../Config/marshaller-explicit.xml              |   2 +-
 .../Examples/ExamplesTest.cs                    |   5 +-
 .../Messaging/MessagingExample.cs               |   9 +-
 .../test/resources/spring-ping-pong-partner.xml |  18 --
 144 files changed, 2234 insertions(+), 1082 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1eaa8c92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1eaa8c92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------


[05/43] ignite git commit: Fixing keepBinary for continuous queries.

Posted by yz...@apache.org.
Fixing keepBinary for continuous queries.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 01c24e7d07e15df8a9a4722c0ec2a9366cb2f669
Parents: eeb9142
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 21:40:52 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 21:40:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |  1 +
 .../internal/GridEventConsumeHandler.java       |  5 ++
 .../internal/GridMessageListenHandler.java      |  5 ++
 .../processors/cache/IgniteCacheProxy.java      | 14 +++--
 .../continuous/CacheContinuousQueryEntry.java   | 50 ++++++++++++-----
 .../continuous/CacheContinuousQueryEvent.java   |  6 +--
 .../continuous/CacheContinuousQueryHandler.java | 33 +++++++++---
 .../CacheContinuousQueryListener.java           |  5 ++
 .../continuous/CacheContinuousQueryManager.java | 57 ++++++++++++--------
 .../continuous/GridContinuousHandler.java       |  5 ++
 .../continuous/GridContinuousProcessor.java     | 10 +++-
 .../StartRoutineDiscoveryMessage.java           | 13 ++++-
 12 files changed, 156 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 5a31415..8733bb3 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry;
 import org.apache.ignite.internal.util.UUIDCollectionMessage;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 3918976..f4bbd6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -125,6 +125,11 @@ class GridEventConsumeHandler implements GridContinuousHandler {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean keepBinary() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String cacheName() {
         throw new IllegalStateException();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
index aa837b8..9a2829b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
@@ -98,6 +98,11 @@ public class GridMessageListenHandler implements GridContinuousHandler {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean keepBinary() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String cacheName() {
         throw new IllegalStateException();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index cb36432..2a52a1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -555,7 +555,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
      * @return Initial iteration cursor.
      */
     @SuppressWarnings("unchecked")
-    private QueryCursor<Cache.Entry<K, V>> queryContinuous(ContinuousQuery qry, boolean loc) {
+    private QueryCursor<Cache.Entry<K, V>> queryContinuous(ContinuousQuery qry, boolean loc, boolean keepBinary) {
         if (qry.getInitialQuery() instanceof ContinuousQuery)
             throw new IgniteException("Initial predicate for continuous query can't be an instance of another " +
                 "continuous query. Use SCAN or SQL query for initial iteration.");
@@ -570,7 +570,8 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                 qry.getPageSize(),
                 qry.getTimeInterval(),
                 qry.isAutoUnsubscribe(),
-                loc);
+                loc,
+                keepBinary);
 
             final QueryCursor<Cache.Entry<K, V>> cur =
                 qry.getInitialQuery() != null ? query(qry.getInitialQuery()) : null;
@@ -616,8 +617,11 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
             validate(qry);
 
+            CacheOperationContext opCtxCall = ctx.operationContextPerCall();
+
             if (qry instanceof ContinuousQuery)
-                return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal());
+                return (QueryCursor<R>)queryContinuous((ContinuousQuery<K, V>)qry, qry.isLocal(),
+                    opCtxCall != null && opCtxCall.isKeepBinary());
 
             if (qry instanceof SqlQuery) {
                 final SqlQuery p = (SqlQuery)qry;
@@ -1623,7 +1627,9 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         CacheOperationContext prev = onEnter(gate, opCtx);
 
         try {
-            ctx.continuousQueries().executeJCacheQuery(lsnrCfg, false);
+            CacheOperationContext opCtx = ctx.operationContextPerCall();
+
+            ctx.continuousQueries().executeJCacheQuery(lsnrCfg, false, opCtx != null && opCtx.isKeepBinary());
         }
         catch (IgniteCheckedException e) {
             throw cacheException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
index 0495e6d..4d3786a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
@@ -99,6 +99,9 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
     /** Filtered events. */
     private GridLongList filteredEvts;
 
+    /** Keep binary. */
+    private boolean keepBinary;
+
     /**
      * Required by {@link Message}.
      */
@@ -122,6 +125,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
         KeyCacheObject key,
         @Nullable CacheObject newVal,
         @Nullable CacheObject oldVal,
+        boolean keepBinary,
         int part,
         long updateCntr,
         @Nullable AffinityTopologyVersion topVer) {
@@ -133,6 +137,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
         this.part = part;
         this.updateCntr = updateCntr;
         this.topVer = topVer;
+        this.keepBinary = keepBinary;
     }
 
     /**
@@ -203,6 +208,13 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
     }
 
     /**
+     * @return Keep binary flag.
+     */
+    boolean isKeepBinary() {
+        return keepBinary;
+    }
+
+    /**
      * @param cntrs Filtered events.
      */
     void filteredEvents(GridLongList cntrs) {
@@ -322,36 +334,42 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeMessage("key", key))
+                if (!writer.writeBoolean("keepBinary", keepBinary))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeMessage("newVal", newVal))
+                if (!writer.writeMessage("key", key))
                     return false;
 
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeMessage("oldVal", oldVal))
+                if (!writer.writeMessage("newVal", newVal))
                     return false;
 
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeInt("part", part))
+                if (!writer.writeMessage("oldVal", oldVal))
                     return false;
 
                 writer.incrementState();
 
             case 8:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeInt("part", part))
                     return false;
 
                 writer.incrementState();
 
             case 9:
+                if (!writer.writeMessage("topVer", topVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 10:
                 if (!writer.writeLong("updateCntr", updateCntr))
                     return false;
 
@@ -407,7 +425,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 4:
-                key = reader.readMessage("key");
+                keepBinary = reader.readBoolean("keepBinary");
 
                 if (!reader.isLastRead())
                     return false;
@@ -415,7 +433,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 5:
-                newVal = reader.readMessage("newVal");
+                key = reader.readMessage("key");
 
                 if (!reader.isLastRead())
                     return false;
@@ -423,7 +441,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 6:
-                oldVal = reader.readMessage("oldVal");
+                newVal = reader.readMessage("newVal");
 
                 if (!reader.isLastRead())
                     return false;
@@ -431,7 +449,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 7:
-                part = reader.readInt("part");
+                oldVal = reader.readMessage("oldVal");
 
                 if (!reader.isLastRead())
                     return false;
@@ -439,7 +457,7 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 8:
-                topVer = reader.readMessage("topVer");
+                part = reader.readInt("part");
 
                 if (!reader.isLastRead())
                     return false;
@@ -447,6 +465,14 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
                 reader.incrementState();
 
             case 9:
+                topVer = reader.readMessage("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 10:
                 updateCntr = reader.readLong("updateCntr");
 
                 if (!reader.isLastRead())
@@ -461,11 +487,11 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 10;
+        return 11;
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheContinuousQueryEntry.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
index d26be5f..f665339 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
@@ -59,18 +59,18 @@ class CacheContinuousQueryEvent<K, V> extends CacheEntryEvent<K, V> {
     /** {@inheritDoc} */
     @Override
     public K getKey() {
-        return (K)cctx.cacheObjectContext().unwrapPortableIfNeeded(e.key(), true, false);
+        return (K)cctx.cacheObjectContext().unwrapPortableIfNeeded(e.key(), e.isKeepBinary(), false);
     }
 
     /** {@inheritDoc} */
     @Override public V getValue() {
-        return (V)cctx.cacheObjectContext().unwrapPortableIfNeeded(e.value(), true, false);
+        return (V)cctx.cacheObjectContext().unwrapPortableIfNeeded(e.value(), e.isKeepBinary(), false);
     }
 
     /** {@inheritDoc} */
     @Override
     public V getOldValue() {
-        return (V)cctx.cacheObjectContext().unwrapPortableIfNeeded(e.oldValue(), true, false);
+        return (V)cctx.cacheObjectContext().unwrapPortableIfNeeded(e.oldValue(), e.isKeepBinary(), false);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index b69d4cd..aa9bea3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -81,7 +81,7 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
 /**
  * Continuous query handler.
  */
-class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
+public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -128,7 +128,10 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
     private transient Collection<CacheContinuousQueryEntry> backupQueue;
 
     /** */
-    private boolean localCache;
+    private boolean locCache;
+
+    /** */
+    private transient boolean keepBinary;
 
     /** */
     private transient ConcurrentMap<Integer, PartitionRecovery> rcvs;
@@ -180,7 +183,8 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
         boolean ignoreExpired,
         int taskHash,
         boolean skipPrimaryCheck,
-        boolean locCache) {
+        boolean locCache,
+        boolean keepBinary) {
         assert topic != null;
         assert locLsnr != null;
 
@@ -195,7 +199,8 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
         this.ignoreExpired = ignoreExpired;
         this.taskHash = taskHash;
         this.skipPrimaryCheck = skipPrimaryCheck;
-        this.localCache = locCache;
+        this.locCache = locCache;
+        this.keepBinary = keepBinary;
 
         cacheId = CU.cacheId(cacheName);
     }
@@ -216,6 +221,18 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean keepBinary() {
+        return keepBinary;
+    }
+
+    /**
+     * @param keepBinary Keep binary flag.
+     */
+    public void keepBinary(boolean keepBinary) {
+        this.keepBinary = keepBinary;
+    }
+
+    /** {@inheritDoc} */
     @Override public String cacheName() {
         return cacheName;
     }
@@ -284,6 +301,11 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
                 }
             }
 
+            /** {@inheritDoc} */
+            @Override public boolean keepBinary() {
+                return keepBinary;
+            }
+
             @Override public void onEntryUpdated(CacheContinuousQueryEvent<K, V> evt, boolean primary,
                 boolean recordIgniteEvt) {
                 if (ignoreExpired && evt.getEventType() == EventType.EXPIRED)
@@ -317,7 +339,7 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
 
                     if (primary || skipPrimaryCheck) {
                         if (loc) {
-                            if (!localCache) {
+                            if (!locCache) {
                                 Collection<CacheContinuousQueryEntry> entries = handleEvent(ctx, entry);
 
                                 if (!entries.isEmpty()) {
@@ -848,7 +870,6 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
 
         /**
          * @param e Entry.
-         * @param topVer Topology version.
          * @return Continuous query entry.
          */
         private CacheContinuousQueryEntry skipEntry(CacheContinuousQueryEntry e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
index 8342acf..86abbef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryListener.java
@@ -81,6 +81,11 @@ interface CacheContinuousQueryListener<K, V> {
     public boolean oldValueRequired();
 
     /**
+     * @return Keep binary flag.
+     */
+    public boolean keepBinary();
+
+    /**
      * @return Whether to notify on existing entries.
      */
     public boolean notifyExisting();

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index b2e7490..0e4cb40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -135,7 +135,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
 
         if (cfgs != null) {
             for (CacheEntryListenerConfiguration cfg : cfgs)
-                executeJCacheQuery(cfg, true);
+                executeJCacheQuery(cfg, true, false);
         }
     }
 
@@ -161,21 +161,23 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      */
     public void skipUpdateEvent(KeyCacheObject key, int partId, long updCntr, AffinityTopologyVersion topVer) {
         if (lsnrCnt.get() > 0) {
-            CacheContinuousQueryEntry e0 = new CacheContinuousQueryEntry(
-                cctx.cacheId(),
-                UPDATED,
-                key,
-                null,
-                null,
-                partId,
-                updCntr,
-                topVer);
+            for (CacheContinuousQueryListener lsnr : lsnrs.values()) {
+                CacheContinuousQueryEntry e0 = new CacheContinuousQueryEntry(
+                    cctx.cacheId(),
+                    UPDATED,
+                    key,
+                    null,
+                    null,
+                    lsnr.keepBinary(),
+                    partId,
+                    updCntr,
+                    topVer);
 
-            CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(
+                CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(
                     cctx.kernalContext().cache().jcache(cctx.name()), cctx, e0);
 
-            for (CacheContinuousQueryListener lsnr : lsnrs.values())
                 lsnr.skipUpdateEvent(evt, topVer);
+            }
         }
     }
 
@@ -253,6 +255,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                 key,
                 newVal,
                 lsnr.oldValueRequired() ? oldVal : null,
+                lsnr.keepBinary(),
                 partId,
                 updateCntr,
                 topVer);
@@ -306,6 +309,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                     key,
                     null,
                     lsnr.oldValueRequired() ? oldVal : null,
+                    lsnr.keepBinary(),
                     e.partition(),
                     -1,
                     null);
@@ -333,7 +337,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         int bufSize,
         long timeInterval,
         boolean autoUnsubscribe,
-        boolean loc) throws IgniteCheckedException
+        boolean loc,
+        boolean keepBinary) throws IgniteCheckedException
     {
         return executeQuery0(
             locLsnr,
@@ -346,7 +351,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             true,
             false,
             true,
-            loc);
+            loc,
+            keepBinary);
     }
 
     /**
@@ -374,7 +380,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             true,
             false,
             true,
-            loc);
+            loc,
+            false);
     }
 
     /**
@@ -395,9 +402,9 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @param onStart Whether listener is created on node start.
      * @throws IgniteCheckedException If failed.
      */
-    public void executeJCacheQuery(CacheEntryListenerConfiguration cfg, boolean onStart)
+    public void executeJCacheQuery(CacheEntryListenerConfiguration cfg, boolean onStart, boolean keepBinary)
         throws IgniteCheckedException {
-        JCacheQuery lsnr = new JCacheQuery(cfg, onStart);
+        JCacheQuery lsnr = new JCacheQuery(cfg, onStart, keepBinary);
 
         JCacheQuery old = jCacheLsnrs.putIfAbsent(cfg, lsnr);
 
@@ -471,7 +478,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         boolean oldValRequired,
         boolean sync,
         boolean ignoreExpired,
-        boolean loc) throws IgniteCheckedException
+        boolean loc,
+        final boolean keepBinary) throws IgniteCheckedException
     {
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -492,7 +500,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
             ignoreExpired,
             taskNameHash,
             skipPrimaryCheck,
-            cctx.isLocal());
+            cctx.isLocal(),
+            keepBinary);
 
         IgnitePredicate<ClusterNode> pred = (loc || cctx.config().getCacheMode() == CacheMode.LOCAL) ?
             F.nodeForNodeId(cctx.localNodeId()) : F.<ClusterNode>alwaysTrue();
@@ -550,6 +559,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                                     e.key(),
                                     e.rawGet(),
                                     null,
+                                    keepBinary,
                                     0,
                                     -1,
                                     null);
@@ -633,15 +643,19 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         private final boolean onStart;
 
         /** */
+        private final boolean keepBinary;
+
+        /** */
         private volatile UUID routineId;
 
         /**
          * @param cfg Listener configuration.
          * @param onStart {@code True} if executed on cache start.
          */
-        private JCacheQuery(CacheEntryListenerConfiguration cfg, boolean onStart) {
+        private JCacheQuery(CacheEntryListenerConfiguration cfg, boolean onStart, boolean keepBinary) {
             this.cfg = cfg;
             this.onStart = onStart;
+            this.keepBinary = keepBinary;
         }
 
         /**
@@ -694,7 +708,8 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                 cfg.isOldValueRequired(),
                 cfg.isSynchronous(),
                 false,
-                false);
+                false,
+                keepBinary);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
index d8698b3..68b83ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
@@ -143,6 +143,11 @@ public interface GridContinuousHandler extends Externalizable, Cloneable {
     public boolean isForQuery();
 
     /**
+     * @return {@code True} if Ignite Binary objects should be passed to the listener and filter.
+     */
+    public boolean keepBinary();
+
+    /**
      * @return Cache name if this is a continuous query handler.
      */
     public String cacheName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index e218790..f473d02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -612,7 +613,8 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             if (locIncluded && registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true))
                 hnd.onListenerRegistered(routineId, ctx);
 
-            ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData));
+            ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData,
+                reqData.handler().keepBinary()));
         }
         catch (IgniteCheckedException e) {
             startFuts.remove(routineId);
@@ -822,6 +824,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         GridContinuousHandler hnd = data.handler();
 
+        if (req.keepBinary()) {
+            assert hnd instanceof CacheContinuousQueryHandler;
+
+            ((CacheContinuousQueryHandler)hnd).keepBinary(true);
+        }
+
         IgniteCheckedException err = null;
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/01c24e7d/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
index 82c0377..ff037d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
@@ -40,14 +40,18 @@ public class StartRoutineDiscoveryMessage extends AbstractContinuousMessage {
     /** */
     private Map<Integer, Long> updateCntrs;
 
+    /** Keep binary flag. */
+    private boolean keepBinary;
+
     /**
      * @param routineId Routine id.
      * @param startReqData Start request data.
      */
-    public StartRoutineDiscoveryMessage(UUID routineId, StartRequestData startReqData) {
+    public StartRoutineDiscoveryMessage(UUID routineId, StartRequestData startReqData, boolean keepBinary) {
         super(routineId);
 
         this.startReqData = startReqData;
+        this.keepBinary = keepBinary;
     }
 
     /**
@@ -88,6 +92,13 @@ public class StartRoutineDiscoveryMessage extends AbstractContinuousMessage {
         return errs;
     }
 
+    /**
+     * @return {@code True} if keep binary flag was set on continuous handler.
+     */
+    public boolean keepBinary() {
+        return keepBinary;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean isMutable() {
         return true;


[12/43] ignite git commit: ignite-1282 - Debugging failover.

Posted by yz...@apache.org.
ignite-1282 - Debugging failover.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 4844b3edafe2ec3d5eaf93ce56a87c45e2b8e950
Parents: 6dc6ffe
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 14:41:08 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 14:41:08 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheIoManager.java      | 2 ++
 .../processors/cache/distributed/dht/GridDhtTxFinishFuture.java   | 1 +
 .../cache/distributed/dht/GridPartitionedSingleGetFuture.java     | 3 ++-
 .../distributed/dht/IgniteCachePutRetryAbstractSelfTest.java      | 2 ++
 4 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4844b3ed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 9afbca8..7b1d749 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -145,6 +145,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
 
             if (fut != null && !fut.isDone()) {
+                U.debug(log, "<> Will wait for affinity ready future [fut=" + fut + ", msg=" + msg + ']');
+
                 fut.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> t) {
                         cctx.kernalContext().closure().runLocalSafe(new Runnable() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4844b3ed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index e8ef5d4..c4a90b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -474,6 +474,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
         });
 
         return S.toString(GridDhtTxFinishFuture.class, this,
+            "xidVer", tx.xidVersion(),
             "innerFuts", futs,
             "super", super.toString());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4844b3ed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index f276cac..a5e5d53 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -323,7 +323,7 @@ public class GridPartitionedSingleGetFuture extends GridFutureAdapter<Object> im
             GridDhtCacheAdapter colocated = cctx.dht();
 
             while (true) {
-                GridCacheEntryEx entry;
+                GridCacheEntryEx entry = null;
 
                 try {
                     entry = colocated.context().isSwapOrOffheapEnabled() ? colocated.entryEx(key) :
@@ -401,6 +401,7 @@ public class GridPartitionedSingleGetFuture extends GridFutureAdapter<Object> im
                     return null;
                 }
                 catch (GridCacheEntryRemovedException ignored) {
+                    U.debug(log, ">>>>>>>>> " + entry);
                     // No-op, will retry.
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4844b3ed/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index ee28cf9..ce09a64 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -122,6 +122,8 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
 
         cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 
+        cfg.setIncludeEventTypes(new int[0]);
+
         return cfg;
     }
 


[26/43] ignite git commit: Fixing getAnd* methods with binary marshaller.

Posted by yz...@apache.org.
Fixing getAnd* methods with binary marshaller.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: ae0ea3cf9adcf73958cae9e9b8183cab907694de
Parents: afcf0ab
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 11:36:08 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 11:36:08 2015 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  3 +-
 .../transactions/IgniteTxLocalAdapter.java      | 29 ++++--
 .../cache/GridCacheAbstractFullApiSelfTest.java | 97 ++++++++++++++++++++
 3 files changed, 120 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae0ea3cf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 07111a1..513e6e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -324,7 +324,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
         GridCacheReturn ret = (GridCacheReturn)res;
 
         Object retval =
-            res == null ? null : rawRetval ? ret : (this.retval || op == TRANSFORM) ? ret.value() : ret.success();
+            res == null ? null : rawRetval ? ret : (this.retval || op == TRANSFORM) ?
+                cctx.unwrapPortableIfNeeded(ret.value(), keepBinary) : ret.success();
 
         if (op == TRANSFORM && retval == null)
             retval = Collections.emptyMap();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae0ea3cf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 7c6a1d4..b3ff3a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -2932,6 +2932,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
             KeyCacheObject cacheKey = cacheCtx.toCacheKeyObject(key);
 
+            boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
+
             final IgniteInternalFuture<Void> loadFut = enlistWrite(
                 cacheCtx,
                 cacheKey,
@@ -2945,7 +2947,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 ret,
                 opCtx != null && opCtx.skipStore(),
                 /*singleRmv*/false,
-                opCtx != null && opCtx.isKeepBinary());
+                keepBinary);
 
             if (pessimistic()) {
                 assert loadFut == null || loadFut.isDone() : loadFut;
@@ -3009,7 +3011,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 }
             }
             else
-                return optimisticPutFuture(loadFut, ret);
+                return optimisticPutFuture(cacheCtx, loadFut, ret, keepBinary);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture(e);
@@ -3099,6 +3101,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
             CacheOperationContext opCtx = cacheCtx.operationContextPerCall();
 
+            final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
+
             final IgniteInternalFuture<Void> loadFut = enlistWrite(
                 cacheCtx,
                 keySet,
@@ -3115,7 +3119,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 null,
                 opCtx != null && opCtx.skipStore(),
                 false,
-                opCtx != null && opCtx.isKeepBinary());
+                keepBinary);
 
             if (pessimistic()) {
                 assert loadFut == null || loadFut.isDone() : loadFut;
@@ -3177,7 +3181,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 }
             }
             else
-                return optimisticPutFuture(loadFut, ret);
+                return optimisticPutFuture(cacheCtx, loadFut, ret, keepBinary);
         }
         catch (RuntimeException e) {
             onException();
@@ -3191,7 +3195,12 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
      * @param ret Future result.
      * @return Future.
      */
-    private IgniteInternalFuture optimisticPutFuture(IgniteInternalFuture<Void> loadFut, final GridCacheReturn ret) {
+    private IgniteInternalFuture optimisticPutFuture(
+        final GridCacheContext cacheCtx,
+        IgniteInternalFuture<Void> loadFut,
+        final GridCacheReturn ret,
+        final boolean keepBinary
+    ) {
         if (implicit()) {
             // Should never load missing values for implicit transaction as values will be returned
             // with prepare response, if required.
@@ -3211,7 +3220,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         try {
                             txFut.get();
 
-                            return implicitRes;
+                            return new GridCacheReturn(cacheCtx, true, keepBinary,
+                                implicitRes.value(), implicitRes.success());
                         }
                         catch (IgniteCheckedException | RuntimeException e) {
                             rollbackAsync();
@@ -3337,6 +3347,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         else
             plc = null;
 
+        final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
+
         final IgniteInternalFuture<Void> loadFut = enlistWrite(
             cacheCtx,
             keys0,
@@ -3353,7 +3365,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
             drMap,
             opCtx != null && opCtx.skipStore(),
             singleRmv,
-            opCtx != null && opCtx.isKeepBinary()
+            keepBinary
         );
 
         if (log.isDebugEnabled())
@@ -3432,7 +3444,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         try {
                             txFut.get();
 
-                            return implicitRes;
+                            return new GridCacheReturn(cacheCtx, true, keepBinary,
+                                implicitRes.value(), implicitRes.success());
                         }
                         catch (IgniteCheckedException | RuntimeException e) {
                             rollbackAsync();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae0ea3cf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 89c4029..c10f81a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -2393,6 +2393,60 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testGetAndRemoveObject() throws Exception {
+        IgniteCache<String, TestValue> cache = ignite(0).cache(null);
+
+        TestValue val1 = new TestValue(1);
+        TestValue val2 = new TestValue(2);
+
+        cache.put("key1", val1);
+        cache.put("key2", val2);
+
+        assert !cache.remove("key1", new TestValue(0));
+
+        TestValue oldVal = cache.get("key1");
+
+        assert oldVal != null && F.eq(val1, oldVal);
+
+        assert cache.remove("key1", val1);
+
+        assert cache.get("key1") == null;
+
+        TestValue oldVal2 = cache.getAndRemove("key2");
+
+        assert F.eq(val2, oldVal2);
+
+        assert cache.get("key2") == null;
+        assert cache.getAndRemove("key2") == null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAndPutObject() throws Exception {
+        IgniteCache<String, TestValue> cache = ignite(0).cache(null);
+
+        TestValue val1 = new TestValue(1);
+        TestValue val2 = new TestValue(2);
+
+        cache.put("key1", val1);
+
+        TestValue oldVal = cache.get("key1");
+
+        assertEquals(val1, oldVal);
+
+        oldVal = cache.getAndPut("key1", val2);
+
+        assertEquals(val1, oldVal);
+
+        TestValue updVal = cache.get("key1");
+
+        assertEquals(val2, updVal);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDeletedEntriesFlag() throws Exception {
         if (cacheMode() != LOCAL && cacheMode() != REPLICATED && memoryMode() != OFFHEAP_TIERED) {
             final int cnt = 3;
@@ -5478,4 +5532,47 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             throw new EntryProcessorException("Test entry processor exception.");
         }
     }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private int val;
+
+        /**
+         * @param val Value.
+         */
+        TestValue(int val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public int value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (!(o instanceof TestValue))
+                return false;
+
+            TestValue value = (TestValue)o;
+
+            if (val != value.val)
+                return false;
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+    }
 }


[38/43] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: e8a14dabe0029e3ff6a4be4e49f409c73e304f9a
Parents: be505f4 e386798
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 12:56:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 12:56:04 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/IgniteBinary.java     | 10 +++++-----
 .../main/java/org/apache/ignite/binary/BinaryObject.java  |  2 +-
 .../main/java/org/apache/ignite/binary/BinaryType.java    |  2 +-
 .../processors/cache/portable/IgniteBinaryImpl.java       |  9 +++++----
 .../internal/processors/cacheobject/NoOpBinary.java       |  8 ++++----
 .../portable/BinaryObjectBuilderAdditionalSelfTest.java   |  6 +++---
 .../internal/portable/GridPortableMetaDataSelfTest.java   | 10 +++++-----
 ...dCacheClientNodeBinaryObjectMetadataMultinodeTest.java |  8 ++++----
 .../GridCacheClientNodeBinaryObjectMetadataTest.java      |  4 ++--
 .../platform/PlatformComputeBinarizableArgTask.java       |  2 +-
 .../Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs     |  5 ++++-
 .../Apache.Ignite.Examples/Messaging/MessagingExample.cs  |  9 +++++++--
 12 files changed, 42 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[32/43] ignite git commit: Added cacheId for toString.

Posted by yz...@apache.org.
Added cacheId for toString.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 72e39c8a67e380c89abd648313634cfdd532123c
Parents: d99fc8d
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 12:34:36 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 12:34:36 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheMessage.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/72e39c8a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 61136bf..89dd226 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -84,6 +84,7 @@ public abstract class GridCacheMessage implements Message {
     private boolean skipPrepare;
 
     /** Cache ID. */
+    @GridToStringInclude
     protected int cacheId;
 
     /**


[42/43] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: c584ccae974cde2221220cc5daade57844f37132
Parents: c3eeb6d dae1b87
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 13:31:03 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 13:31:03 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheIoManager.java  | 2 --
 .../ignite/internal/processors/cache/GridCacheProcessor.java  | 7 -------
 .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 1 -
 3 files changed, 10 deletions(-)
----------------------------------------------------------------------



[11/43] ignite git commit: Merge branch ignite-1.5 into ignite-1282

Posted by yz...@apache.org.
Merge branch ignite-1.5 into ignite-1282


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 6dc6ffe12d6587a2b8b1dcc497828f16be45aa19
Parents: e52b267 eee3b21
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 12:39:07 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 12:39:07 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteBinary.java    |  2 +-
 .../internal/portable/BinaryTypeImpl.java       |  4 ++-
 .../processors/cache/CacheObjectContext.java    | 26 +++++++++++++++---
 .../datastructures/GridCacheAtomicLongImpl.java |  6 +++-
 .../portable/BinaryMarshallerSelfTest.java      |  6 ++--
 .../examples/project/vs/ignite-examples.vcxproj |  6 ++--
 .../Apache.Ignite.Benchmarks.csproj             |  3 ++
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |  2 ++
 .../Apache.Ignite.Core.Tests.csproj             |  2 ++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 29 ++++++++++++--------
 .../Apache.Ignite.Core.csproj                   |  2 ++
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  8 ++++--
 modules/platforms/dotnet/Apache.Ignite.sln      |  6 ++--
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |  2 ++
 .../Apache.Ignite.Examples.csproj               |  2 ++
 .../Apache.Ignite.ExamplesDll.csproj            |  2 ++
 16 files changed, 77 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6dc6ffe1/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6dc6ffe1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6dc6ffe1/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------


[08/43] ignite git commit: Fixing tests.

Posted by yz...@apache.org.
Fixing tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 9922d83b8de24f70a89913bdf0c9739cc6dd9d35
Parents: a129439
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 11:06:26 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 11:06:26 2015 +0300

----------------------------------------------------------------------
 .../store/jdbc/CacheAbstractJdbcStore.java      |  2 +-
 .../configuration/CacheConfiguration.java       |  5 +-
 .../processors/cache/GridCacheMessage.java      |  2 +-
 .../processors/cache/GridCacheProcessor.java    |  9 ++-
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java | 85 ++++++++++++++++++++
 ...JdbcPojoStorePortableMarshallerSelfTest.java | 85 --------------------
 .../ignite/testsuites/IgniteCacheTestSuite.java |  4 +-
 7 files changed, 99 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
index 6dc413b..7617e48 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
@@ -68,7 +68,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lifecycle.LifecycleAware;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.resources.CacheStoreSessionResource;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 8c3ea19..26bfdbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -209,7 +209,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /** Default value for keep binary in store behavior .*/
     @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"})
-    public static final Boolean DFLT_KEEP_BINARY_IN_STORE = new Boolean(true);
+    public static final Boolean DFLT_KEEP_BINARY_IN_STORE = new Boolean(false);
 
     /** Default threshold for concurrent loading of keys from {@link CacheStore}. */
     public static final int DFLT_CONCURRENT_LOAD_ALL_THRESHOLD = 5;
@@ -888,8 +888,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Flag indicating that {@link CacheStore} implementation
      * is working with binary objects instead of Java objects.
-     * Default value of this flag is {@link #DFLT_KEEP_BINARY_IN_STORE},
-     * because this is recommended behavior from performance standpoint.
+     * Default value of this flag is {@link #DFLT_KEEP_BINARY_IN_STORE}.
      * <p>
      * If set to {@code false}, Ignite will deserialize keys and
      * values stored in binary format before they are passed

http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 61136bf..177454c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -681,6 +681,6 @@ public abstract class GridCacheMessage implements Message {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridCacheMessage.class, this);
+        return S.toString(GridCacheMessage.class, this, "cacheId", cacheId);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 178c5f0..6822ded 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1021,7 +1021,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cfg.isKeepBinaryInStore() && cfg.isKeepBinaryInStore() != CacheConfiguration.DFLT_KEEP_BINARY_IN_STORE
             && !(ctx.config().getMarshaller() instanceof BinaryMarshaller))
             U.warn(log, "CacheConfiguration.isKeepBinaryInStore() configuration property will be ignored because " +
-                "PortableMarshaller is not used");
+                "BinaryMarshaller is not used");
 
         // Start managers.
         for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
@@ -1049,6 +1049,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         cacheCtx.onStarted();
 
+        U.debug(log, "Started cache [name=" + U.maskName(cfg.getName()) + ", deploymentId=" +
+            cacheCtx.dynamicDeploymentId() + ']');
+
         if (log.isInfoEnabled())
             log.info("Started cache [name=" + U.maskName(cfg.getName()) + ", mode=" + cfg.getCacheMode() + ']');
     }
@@ -1601,6 +1604,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
             return;
 
+        U.debug(log, "prepare cache start [locNodeId=" + ctx.localNodeId() +
+            ", initiatingNodeId=" + initiatingNodeId + ", deploymentId=" + deploymentId + ", topVer=" + topVer +
+            ", name=" + cfg.getName() + ']');
+
         if (affNodeStart || clientNodeStart) {
             if (clientNodeStart && !affNodeStart) {
                 if (nearCfg != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
new file mode 100644
index 0000000..659efb0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.jdbc;
+
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
+
+/**
+ * Class for {@code PojoCacheStore} tests.
+ */
+public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoStoreAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Marshaller marshaller(){
+        return new BinaryMarshaller();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheNoKeyClasses() throws Exception {
+        startTestGrid(false, true, false, false);
+
+        checkCacheContent();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheNoKeyClassesTx() throws Exception {
+        startTestGrid(false, true, false, true);
+
+        checkCacheContent();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheNoValueClasses() throws Exception {
+        startTestGrid(false, false, true, false);
+
+        checkCacheContent();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheNoValueClassesTx() throws Exception {
+        startTestGrid(false, false, true, true);
+
+        checkCacheContent();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheNoKeyAndValueClasses() throws Exception {
+        startTestGrid(false, true, true, false);
+
+        checkCacheContent();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheNoKeyAndValueClassesTx() throws Exception {
+        startTestGrid(false, true, true, true);
+
+        checkCacheContent();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStorePortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStorePortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStorePortableMarshallerSelfTest.java
deleted file mode 100644
index 39504b1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStorePortableMarshallerSelfTest.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.jdbc;
-
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
-
-/**
- * Class for {@code PojoCacheStore} tests.
- */
-public class CacheJdbcPojoStorePortableMarshallerSelfTest extends CacheJdbcPojoStoreAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected Marshaller marshaller(){
-        return new BinaryMarshaller();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheNoKeyClasses() throws Exception {
-        startTestGrid(false, true, false, false);
-
-        checkCacheContent();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheNoKeyClassesTx() throws Exception {
-        startTestGrid(false, true, false, true);
-
-        checkCacheContent();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheNoValueClasses() throws Exception {
-        startTestGrid(false, false, true, false);
-
-        checkCacheContent();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheNoValueClassesTx() throws Exception {
-        startTestGrid(false, false, true, true);
-
-        checkCacheContent();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheNoKeyAndValueClasses() throws Exception {
-        startTestGrid(false, true, true, false);
-
-        checkCacheContent();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheNoKeyAndValueClassesTx() throws Exception {
-        startTestGrid(false, true, true, true);
-
-        checkCacheContent();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9922d83b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 8af9443..de1c9c4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -32,7 +32,7 @@ import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreMultitreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
-import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStorePortableMarshallerSelfTest;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
@@ -214,7 +214,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheJdbcBlobStoreMultithreadedSelfTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerSelfTest.class);
-        suite.addTestSuite(CacheJdbcPojoStorePortableMarshallerSelfTest.class);
+        suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerSelfTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreMultitreadedSelfTest.class);
         suite.addTestSuite(GridCacheBalancingStoreSelfTest.class);
         suite.addTestSuite(GridCacheAffinityApiSelfTest.class);


[04/43] ignite git commit: Fixed deployment test.

Posted by yz...@apache.org.
Fixed deployment test.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: eeb91429f09c3525779e2d67e9e1bf4f24da72d1
Parents: 3ca58d7
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 20:03:50 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 20:03:50 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/CacheObjectContext.java    |  2 +-
 .../processors/cache/GridCacheDeploymentManager.java     |  6 +++++-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java           |  2 +-
 .../cache/portable/CacheObjectBinaryProcessorImpl.java   | 11 ++++++++++-
 4 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eeb91429/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
index d49a029..928a662 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
@@ -238,7 +238,7 @@ import org.apache.ignite.internal.util.typedef.F;
             CacheObject co = (CacheObject)o;
 
             if (!keepPortable || co.isPlatformType())
-                return unwrapPortable(co.value(this, true), keepPortable, cpy);
+                return unwrapPortable(co.value(this, cpy), keepPortable, cpy);
         }
 
         return o;

http://git-wip-us.apache.org/repos/asf/ignite/blob/eeb91429/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index 35e8b75..d7f7521 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -179,8 +179,12 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
     /**
      * @param ignore {@code True} to ignore.
      */
-    public void ignoreOwnership(boolean ignore) {
+    public boolean ignoreOwnership(boolean ignore) {
+        boolean old = ignoreOwnership.get();
+
         ignoreOwnership.set(ignore);
+
+        return old;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/eeb91429/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index a8807e1..b3155a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -1049,6 +1049,6 @@ public class GridDhtAtomicUpdateRequest extends GridCacheMessage implements Grid
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridDhtAtomicUpdateRequest.class, this);
+        return S.toString(GridDhtAtomicUpdateRequest.class, this, "super", super.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/eeb91429/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 483b6ce..dd9d43f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -217,7 +217,16 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
-        metaDataCache = (IgniteCacheProxy)ctx.cache().jcache(CU.UTILITY_CACHE_NAME).withNoRetries();
+        IgniteCacheProxy<Object, Object> proxy = ctx.cache().jcache(CU.UTILITY_CACHE_NAME);
+
+        boolean old = proxy.context().deploy().ignoreOwnership(true);
+
+        try {
+            metaDataCache = (IgniteCacheProxy)proxy.withNoRetries();
+        }
+        finally {
+            proxy.context().deploy().ignoreOwnership(old);
+        }
 
         if (clientNode) {
             assert !metaDataCache.context().affinityNode();


[41/43] ignite git commit: Simplified GridDiscoveryManager.resolveDiscoCache (with previous code observed on TC NoSuchElementException from Collections.min(discoCacheHist.entrySet()).

Posted by yz...@apache.org.
Simplified GridDiscoveryManager.resolveDiscoCache (with previous code observed on TC NoSuchElementException from Collections.min(discoCacheHist.entrySet()).


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: c3eeb6dc2ded5a22d80078b031a305b8c9f06805
Parents: e8a14da
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 13:30:36 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 13:30:36 2015 +0300

----------------------------------------------------------------------
 .../managers/discovery/GridDiscoveryManager.java | 19 +++++++------------
 1 file changed, 7 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c3eeb6dc/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 4880338..92d66d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -43,6 +43,7 @@ import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
@@ -175,14 +176,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
     };
 
-    /** Disco history entries comparator. */
-    private static final Comparator<Map.Entry<AffinityTopologyVersion, DiscoCache>> histCmp =
-        new Comparator<Map.Entry<AffinityTopologyVersion, DiscoCache>>() {
-            @Override public int compare(Map.Entry<AffinityTopologyVersion, DiscoCache> o1, Map.Entry<AffinityTopologyVersion, DiscoCache> o2) {
-                return o1.getKey().compareTo(o2.getKey());
-            }
-        };
-
     /** Discovery event worker. */
     private final DiscoveryWorker discoWrk = new DiscoveryWorker();
 
@@ -208,7 +201,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     private final AtomicBoolean lastSegChkRes = new AtomicBoolean(true);
 
     /** Topology cache history. */
-    private final Map<AffinityTopologyVersion, DiscoCache> discoCacheHist =
+    private final ConcurrentNavigableMap<AffinityTopologyVersion, DiscoCache> discoCacheHist =
         new GridBoundedConcurrentOrderedMap<>(DISCOVERY_HISTORY_SIZE);
 
     /** Topology snapshots history. */
@@ -1638,10 +1631,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         if (cache == null) {
             // Find the eldest acceptable discovery cache.
-            Map.Entry<AffinityTopologyVersion, DiscoCache> eldest = Collections.min(discoCacheHist.entrySet(), histCmp);
+            Map.Entry<AffinityTopologyVersion, DiscoCache> eldest = discoCacheHist.firstEntry();
 
-            if (topVer.compareTo(eldest.getKey()) < 0)
-                cache = eldest.getValue();
+            if (eldest != null) {
+                if (topVer.compareTo(eldest.getKey()) < 0)
+                    cache = eldest.getValue();
+            }
         }
 
         if (cache == null) {


[33/43] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: f2092e7eae9b0bbb8d259e946d188170ae961f9b
Parents: 72e39c8 bfb839c
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 12:35:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 12:35:20 2015 +0300

----------------------------------------------------------------------
 .../config/binary/example-ignite-binary.xml     |  44 ----
 examples/config/example-ignite.xml              |  10 +-
 .../ignite/examples/binary/EmployeeKey.java     |   3 +
 .../binary/ExampleBinaryNodeStartup.java        |  36 ---
 ...ComputeClientBinaryTaskExecutionExample.java |   6 +-
 .../CacheClientBinaryPutGetExample.java         |   6 +-
 .../datagrid/CacheClientBinaryQueryExample.java |   6 +-
 .../scalar/examples/ScalarCacheExample.scala    |  19 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   1 +
 .../java/org/apache/ignite/IgniteBinary.java    |   5 +-
 .../java/org/apache/ignite/IgniteCache.java     |   3 +-
 .../apache/ignite/binary/BinaryIdMapper.java    |   3 +-
 .../org/apache/ignite/binary/BinaryObject.java  |   7 -
 .../org/apache/ignite/binary/BinaryType.java    |   7 +
 .../apache/ignite/cache/CacheTypeMetadata.java  |   2 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |   2 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   3 +-
 .../configuration/CacheConfiguration.java       |   6 +-
 .../internal/GridEventConsumeHandler.java       |   5 +
 .../internal/GridMessageListenHandler.java      |   5 +
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   6 +-
 .../ignite/internal/NodeStoppingException.java  |  35 +++
 .../internal/portable/BinaryFieldImpl.java      |   6 +-
 .../internal/portable/BinaryMarshaller.java     | 144 +++++++++++
 .../internal/portable/BinaryObjectEx.java       | 225 +----------------
 .../internal/portable/BinaryObjectExImpl.java   | 245 +++++++++++++++++++
 .../internal/portable/BinaryObjectImpl.java     |   2 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryTypeImpl.java       |   5 +
 .../portable/PortableClassDescriptor.java       |   2 +-
 .../internal/portable/PortableContext.java      |  35 ++-
 .../ignite/internal/portable/PortableUtils.java |   2 +-
 .../builder/BinaryObjectBuilderImpl.java        |   9 +-
 .../builder/PortableBuilderSerializer.java      |   4 +-
 .../processors/cache/CacheObjectContext.java    |   2 +-
 .../cache/GridCacheDeploymentManager.java       |   6 +-
 .../processors/cache/GridCacheEventManager.java |  38 ++-
 .../processors/cache/GridCacheIoManager.java    |   6 +
 .../processors/cache/GridCacheMessage.java      |   2 +-
 .../processors/cache/GridCacheMvccManager.java  |   3 +-
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../processors/cache/IgniteCacheProxy.java      |  14 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |  19 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   1 +
 .../dht/GridPartitionedSingleGetFuture.java     |   3 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   2 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   3 +-
 .../CacheObjectBinaryProcessorImpl.java         |  33 ++-
 .../continuous/CacheContinuousQueryEntry.java   |  50 +++-
 .../continuous/CacheContinuousQueryEvent.java   |   6 +-
 .../continuous/CacheContinuousQueryHandler.java |  32 ++-
 .../CacheContinuousQueryListener.java           |   5 +
 .../continuous/CacheContinuousQueryManager.java |  57 +++--
 .../cache/store/CacheOsStoreManager.java        |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  29 ++-
 .../continuous/GridContinuousHandler.java       |   5 +
 .../continuous/GridContinuousProcessor.java     |  10 +-
 .../StartRoutineDiscoveryMessage.java           |  13 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   2 +-
 .../processors/query/GridQueryProcessor.java    |  28 ++-
 .../marshaller/portable/BinaryMarshaller.java   | 146 -----------
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |  85 +++++++
 ...JdbcPojoStorePortableMarshallerSelfTest.java |  85 -------
 .../portable/BinaryFieldsAbstractSelfTest.java  |  13 +-
 .../portable/BinaryFieldsHeapSelfTest.java      |   4 +-
 .../portable/BinaryFieldsOffheapSelfTest.java   |   3 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   5 +-
 .../BinaryFooterOffsetsHeapSelfTest.java        |   4 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   3 +-
 .../portable/BinaryMarshallerSelfTest.java      |  17 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   1 -
 .../portable/BinaryObjectBuilderSelfTest.java   | 102 +++++---
 .../GridPortableAffinityKeySelfTest.java        |  13 +-
 ...idPortableMarshallerCtxDisabledSelfTest.java |   1 -
 .../portable/GridPortableMetaDataSelfTest.java  |   1 -
 .../portable/GridPortableWildcardsSelfTest.java |   1 -
 .../cache/GridCacheAbstractFullApiSelfTest.java |  97 ++++++++
 ...IgniteCacheAbstractExecutionContextTest.java |   2 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |   2 +
 ...naryObjectsAbstractDataStreamerSelfTest.java |   2 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |   2 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |   2 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   2 +-
 .../GridCachePortableStoreAbstractSelfTest.java |   2 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   2 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |   2 +-
 .../DataStreamProcessorPortableSelfTest.java    |   2 +-
 .../GridDataStreamerImplSelfTest.java           |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 .../GridCacheMemoryModePortableSelfTest.java    |   2 +-
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   2 +-
 ...eapTieredEvictionAtomicPortableSelfTest.java |   2 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |   2 +-
 .../GridCacheOffHeapTieredPortableSelfTest.java |   2 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |   2 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   2 +-
 .../PlatformComputeBinarizableArgTask.java      |   6 +-
 .../ignite/testframework/junits/IgniteMock.java |  39 ++-
 .../IgniteBinaryObjectsCacheTestSuite2.java     |  33 +++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   2 +-
 .../IgnitePortableCacheTestSuite.java           |   2 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  23 ++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   2 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   2 -
 .../IgnitePortableCacheQueryTestSuite.java      |   2 +-
 .../Config/Compute/compute-grid1.xml            |   5 -
 .../Config/marshaller-explicit.xml              |   2 +-
 .../test/resources/spring-ping-pong-partner.xml |  18 --
 113 files changed, 1244 insertions(+), 807 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f2092e7e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------


[18/43] ignite git commit: ignite-1282 - Fixing tests.

Posted by yz...@apache.org.
ignite-1282 - Fixing tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 9c1b7656a39496f03950158bc7fc2305eacdc6d1
Parents: 22326cc
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Nov 25 12:25:47 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Nov 25 12:25:47 2015 +0300

----------------------------------------------------------------------
 .../builder/BinaryObjectBuilderImpl.java        |  9 ++----
 .../processors/query/GridQueryProcessor.java    | 26 +++++++++++------
 .../portable/BinaryObjectBuilderSelfTest.java   | 30 ++++++++++++++++++++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  2 +-
 4 files changed, 51 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1b7656/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
index b2fb7d8..d330c72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/BinaryObjectBuilderImpl.java
@@ -31,7 +31,6 @@ import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableSchema;
 import org.apache.ignite.internal.portable.PortableSchemaRegistry;
 import org.apache.ignite.internal.portable.PortableUtils;
-import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -492,15 +491,15 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
     }
 
     /** {@inheritDoc} */
-    @Override public BinaryObjectBuilder setField(String name, Object val) {
-        GridArgumentCheck.notNull(val, name);
+    @Override public BinaryObjectBuilder setField(String name, Object val0) {
+        Object val = val0 == null ? new PortableValueWithType(PortableUtils.typeByClass(Object.class), null) : val0;
 
         if (assignedVals == null)
             assignedVals = new LinkedHashMap<>();
 
         Object oldVal = assignedVals.put(name, val);
 
-        if (oldVal instanceof PortableValueWithType) {
+        if (oldVal instanceof PortableValueWithType && val0 != null) {
             ((PortableValueWithType)oldVal).value(val);
 
             assignedVals.put(name, oldVal);
@@ -514,8 +513,6 @@ public class BinaryObjectBuilderImpl implements BinaryObjectBuilder {
         if (assignedVals == null)
             assignedVals = new LinkedHashMap<>();
 
-        //int fldId = ctx.fieldId(typeId, fldName);
-
         assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
 
         return this;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1b7656/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 c560e4e..90bb9a6 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
@@ -1546,28 +1546,36 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
 
-            String[] mtdNames = new String[] {bld.toString(), prop};
-
             ClassProperty tmp = null;
 
-            for (String mtdName : mtdNames) {
+            try {
+                tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias);
+            }
+            catch (NoSuchMethodException ignore) {
+                // No-op.
+            }
+
+            if (tmp == null) {
                 try {
-                    tmp = new ClassProperty(cls.getMethod(mtdName), key, alias);
+                    tmp = new ClassProperty(cls.getDeclaredField(prop), key, alias);
                 }
-                catch (NoSuchMethodException ignore) {
-                    // No-op, will try another method or field.
+                catch (NoSuchFieldException ignored) {
+                    // No-op.
                 }
             }
 
             if (tmp == null) {
                 try {
-                    tmp = new ClassProperty(cls.getDeclaredField(prop), key, alias);
+                    tmp = new ClassProperty(cls.getMethod(prop), key, alias);
                 }
-                catch (NoSuchFieldException ignored) {
-                    return null;
+                catch (NoSuchMethodException ignored) {
+                    // No-op.
                 }
             }
 
+            if (tmp == null)
+                return null;
+
             tmp.parent(res);
 
             cls = tmp.type();

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1b7656/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
index a57ed0d..eb16939 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
@@ -125,6 +125,36 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testNullField() throws Exception {
+        BinaryObjectBuilder builder = builder("Class");
+
+        builder.hashCode(42);
+
+        builder.setField("objField", (Object)null);
+
+        builder.setField("otherField", "value");
+
+        BinaryObject obj = builder.build();
+
+        assertNull(obj.field("objField"));
+        assertEquals("value", obj.field("otherField"));
+        assertEquals(42, obj.hashCode());
+
+        builder = builder(obj);
+
+        builder.setField("objField", "value");
+        builder.setField("otherField", (Object)null);
+
+        obj = builder.build();
+
+        assertNull(obj.field("otherField"));
+        assertEquals("value", obj.field("objField"));
+        assertEquals(42, obj.hashCode());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testByteField() throws Exception {
         BinaryObjectBuilder builder = builder("Class");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1b7656/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index f3fbf15..e324302 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -1372,7 +1372,7 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
         /**
          * @return Salary.
          */
-        public double salary() {
+        public int salary() {
             return salary;
         }
 


[19/43] ignite git commit: ignite-1282 - Fixing tests.

Posted by yz...@apache.org.
ignite-1282 - Fixing tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 809a44260d10fb394767ebb1df66857c22cc8cbd
Parents: 9c1b765
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Nov 25 15:16:23 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Nov 25 15:16:23 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheTypeMetadata.java  |  2 ++
 .../processors/cache/GridCacheEventManager.java | 38 ++++++++++++++++++--
 2 files changed, 37 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/809a4426/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
index 2b7205b..76dea6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
@@ -33,7 +33,9 @@ import org.apache.ignite.lang.IgniteBiTuple;
 
 /**
  * Cache type metadata need for configuration of indexes or automatic persistence.
+ * @deprecated Use {@link org.apache.ignite.cache.QueryEntity} instead.
  */
+@Deprecated
 public class CacheTypeMetadata implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;

http://git-wip-us.apache.org/repos/asf/ignite/blob/809a4426/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index afca43b..7ff3a99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -39,6 +39,9 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
  * Cache event manager.
  */
 public class GridCacheEventManager extends GridCacheManagerAdapter {
+    /** Force keep binary flag. Will be set if event notification encountered exception during unmarshalling. */
+    private boolean forceKeepBinary;
+
     /**
      * Adds local event listener.
      *
@@ -262,6 +265,35 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
                     "(try to increase topology history size configuration property of configured " +
                     "discovery SPI): " + evtNodeId);
 
+            keepPortable = keepPortable || forceKeepBinary;
+
+            Object key0;
+            Object val0;
+            Object oldVal0;
+
+            try {
+                key0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepPortable, false);
+                val0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(newVal, keepPortable, false);
+                oldVal0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(oldVal, keepPortable, false);
+            }
+            catch (Exception e) {
+                if (!cctx.cacheObjectContext().processor().isPortableEnabled(cctx.config()))
+                    throw e;
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to unmarshall cache object value for the event notification: " + e);
+
+                if (!forceKeepBinary)
+                    LT.warn(log, null, "Failed to unmarshall cache object value for the event notification " +
+                        "(all further notifications will keep binary object format).");
+
+                forceKeepBinary = true;
+
+                key0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(key, true, false);
+                val0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(newVal, true, false);
+                oldVal0 = cctx.cacheObjectContext().unwrapPortableIfNeeded(oldVal, true, false);
+            }
+
             cctx.gridEvents().record(new CacheEvent(cctx.name(),
                 cctx.localNode(),
                 evtNode,
@@ -269,12 +301,12 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
                 type,
                 part,
                 cctx.isNear(),
-                cctx.cacheObjectContext().unwrapPortableIfNeeded(key, keepPortable, false),
+                key0,
                 xid,
                 lockId,
-                cctx.cacheObjectContext().unwrapPortableIfNeeded(newVal, keepPortable, false),
+                val0,
                 hasNewVal,
-                cctx.cacheObjectContext().unwrapPortableIfNeeded(oldVal, keepPortable, false),
+                oldVal0,
                 hasOldVal,
                 subjId,
                 cloClsName,


[06/43] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282

Posted by yz...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: a129439413b8ad440c70d339b5bc1c4b250d1232
Parents: 01c24e7 e4109f9
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 09:52:05 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 09:52:05 2015 +0300

----------------------------------------------------------------------
 examples/schema-import/bin/db-init.sql          |    3 +-
 .../org/apache/ignite/schema/CacheConfig.java   |    7 +-
 .../java/org/apache/ignite/schema/Demo.java     |   20 +-
 .../org/apache/ignite/cache/QueryIndex.java     |   53 +-
 .../store/jdbc/CacheAbstractJdbcStore.java      |  638 +++++++----
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |   14 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  444 +++++---
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |  277 ++++-
 .../ignite/cache/store/jdbc/JdbcType.java       |  255 +++++
 .../cache/store/jdbc/JdbcTypeDefaultHasher.java |   43 +
 .../ignite/cache/store/jdbc/JdbcTypeField.java  |  172 +++
 .../ignite/cache/store/jdbc/JdbcTypeHasher.java |   34 +
 .../internal/GridEventConsumeHandler.java       |    6 +-
 .../internal/GridMessageListenHandler.java      |    6 +-
 .../internal/portable/BinaryObjectImpl.java     |   32 +-
 .../portable/BinaryObjectOffheapImpl.java       |   27 +-
 .../internal/portable/BinaryReaderExImpl.java   | 1015 ++----------------
 .../portable/BinaryReaderHandlesHolder.java     |   46 +
 .../portable/BinaryReaderHandlesHolderImpl.java |   44 +
 .../portable/GridPortableMarshaller.java        |   23 +-
 .../portable/PortableClassDescriptor.java       |    8 +-
 .../ignite/internal/portable/PortableUtils.java |  861 ++++++++++++++-
 .../portable/builder/PortableBuilderReader.java |   21 +-
 .../processors/cache/GridCacheMapEntry.java     |    3 +-
 .../continuous/CacheContinuousQueryHandler.java |    6 +-
 .../continuous/GridContinuousHandler.java       |    6 +-
 .../continuous/GridContinuousProcessor.java     |   50 +-
 .../platform/PlatformContextImpl.java           |    4 +-
 .../processors/query/GridQueryProcessor.java    |    6 +-
 .../ignite/internal/visor/cache/VisorCache.java |    4 +-
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |  395 +++++++
 ...dbcPojoStoreOptimizedMarshallerSelfTest.java |   31 +
 ...JdbcPojoStorePortableMarshallerSelfTest.java |   85 ++
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  200 ++--
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |    2 +-
 ...chePartitionedAtomicSetFailoverSelfTest.java |    5 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |    2 +-
 .../continuous/GridEventConsumeSelfTest.java    |    2 +-
 .../platform/PlatformComputeEchoTask.java       |   18 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    6 +-
 .../Binary/BinarySelfTest.cs                    |   35 +
 .../Compute/ComputeApiTest.cs                   |   51 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |    2 +-
 .../Impl/Binary/BinaryUtils.cs                  |   36 +-
 .../Impl/Binary/BinaryWriter.cs                 |   32 +-
 modules/schema-import/README.txt                |  176 +--
 .../ignite/schema/generator/CodeGenerator.java  |  281 ++---
 .../ignite/schema/generator/XmlGenerator.java   |  101 +-
 .../apache/ignite/schema/model/IndexItem.java   |   54 -
 .../ignite/schema/model/PojoDescriptor.java     |   72 +-
 .../ignite/schema/model/SchemaDescriptor.java   |    6 +-
 .../schema/parser/DatabaseMetadataParser.java   |   12 +-
 .../apache/ignite/schema/parser/DbTable.java    |   37 +-
 .../parser/dialect/DatabaseMetadataDialect.java |   32 +-
 .../parser/dialect/JdbcMetadataDialect.java     |   22 +-
 .../parser/dialect/OracleMetadataDialect.java   |   24 +-
 .../apache/ignite/schema/ui/ModalDialog.java    |    6 +-
 .../ignite/schema/ui/SchemaImportApp.java       |   13 +-
 .../schema/test/AbstractSchemaImportTest.java   |    4 +-
 .../schema/test/model/ignite-type-metadata.xml  |  610 ++++++-----
 .../yardstick/config/ignite-store-config.xml    |   50 +-
 .../IgniteAtomicInvokeRetryBenchmark.java       |    2 +-
 .../failover/IgniteConsistencyException.java    |   64 ++
 ...IgniteTransactionalInvokeRetryBenchmark.java |    2 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java |  135 ++-
 .../IgniteTransactionalWriteReadBenchmark.java  |    2 +-
 66 files changed, 4336 insertions(+), 2399 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a1294394/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1294394/modules/core/src/main/java/org/apache/ignite/internal/GridMessageListenHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1294394/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1294394/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1294394/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
index 68b83ea,3d6e266..900835a
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
@@@ -140,14 -140,9 +140,14 @@@ public interface GridContinuousHandler 
      /**
       * @return {@code True} if for continuous queries.
       */
-     public boolean isForQuery();
+     public boolean isQuery();
  
      /**
 +     * @return {@code True} if Ignite Binary objects should be passed to the listener and filter.
 +     */
 +    public boolean keepBinary();
 +
 +    /**
       * @return Cache name if this is a continuous query handler.
       */
      public String cacheName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1294394/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------


[09/43] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282

Posted by yz...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: e52b267bdf09be4b1661fe2d751e5505cac941ad
Parents: 9922d83 dcbfbd2
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 11:06:56 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 11:06:56 2015 +0300

----------------------------------------------------------------------
 .../Continuous/ContinuousQueryAbstractTest.cs   | 26 ++++++++++++--------
 1 file changed, 16 insertions(+), 10 deletions(-)
----------------------------------------------------------------------



[30/43] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 5cf8daee9785f9a754394c675858935e649bbc38
Parents: 58b2944 d99fc8d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 26 12:01:09 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 26 12:01:09 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------



[17/43] ignite git commit: IGNITE-1282 Added support for IgniteBinary to IgniteMock.

Posted by yz...@apache.org.
IGNITE-1282 Added support for IgniteBinary to IgniteMock.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 22326cc9e7994cbb4bf66471503d47ea4f5d4c00
Parents: 0f1dd6f
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Nov 25 15:17:01 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Nov 25 15:17:01 2015 +0700

----------------------------------------------------------------------
 .../ignite/testframework/junits/IgniteMock.java | 39 +++++++++++++++++++-
 1 file changed, 38 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/22326cc9/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 27a7051..7f27f36 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
@@ -42,12 +42,18 @@ import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.portable.BinaryCachingMetadataHandler;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.internal.portable.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.processors.cacheobject.NoOpBinary;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.IgnitePlugin;
@@ -79,6 +85,12 @@ public class IgniteMock implements Ignite {
     /** */
     private IgniteConfiguration staticCfg;
 
+    /** */
+    private IgniteBinary binaryMock;
+
+    /** */
+    private PortableContext ctx;
+
     /**
      * Mock values
      *
@@ -278,12 +290,37 @@ public class IgniteMock implements Ignite {
 
     /** {@inheritDoc} */
     @Override public IgniteBinary binary() {
-        return null;
+        if (binaryMock != null)
+            return binaryMock;
+
+        if (ctx == null) {
+            /** {@inheritDoc} */
+            ctx = new PortableContext(BinaryCachingMetadataHandler.create(), configuration()) {
+                @Override public int typeId(String typeName) {
+                    return typeName.hashCode();
+                }
+            };
+        }
+
+        binaryMock = new NoOpBinary() {
+            /** {@inheritDoc} */
+            @Override public int typeId(String typeName) {
+                return typeName.hashCode();
+            }
+
+            /** {@inheritDoc} */
+            @Override public BinaryObjectBuilder builder(String typeName) throws BinaryObjectException {
+                return new BinaryObjectBuilderImpl(ctx, typeName);
+            }
+        };
+
+        return binaryMock;
     }
 
     /** {@inheritDoc} */
     @Override public void close() {}
 
+    /** {@inheritDoc} */
     @Nullable @Override public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) {
         return null;
     }


[10/43] ignite git commit: IGNITE-1282 Debug tests.

Posted by yz...@apache.org.
IGNITE-1282 Debug tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: b75184304b03d8a7cba24473b550669ec95f7512
Parents: 1e732a0
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Nov 24 15:15:04 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 24 15:15:04 2015 +0700

----------------------------------------------------------------------
 .../scalar/examples/ScalarCacheExample.scala     | 19 +++++++++++++++----
 1 file changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b7518430/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala
index 0bf8d6f..32afab2 100644
--- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala
+++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala
@@ -49,6 +49,10 @@ object ScalarCacheExample extends App {
 
             basicOperations()
         }
+        catch {
+            case e: Throwable =>
+                e.printStackTrace();
+        }
         finally {
             cache.destroy()
         }
@@ -78,13 +82,20 @@ object ScalarCacheExample extends App {
         // Put one more value.
         c += (3.toString -> 11)
 
-        // Get with option...
-        c.opt(44.toString) match {
-            case Some(v) => sys.error("Should never happen.")
-            case None => println("Correct")
+        try {
+            c.opt(44.toString) match {
+                case Some(v) => sys.error("Should never happen.")
+                case _ => println("Correct")
+            }
         }
+        catch {
+            case e: Throwable =>
+                e.printStackTrace()
+        }
+
 
         // Print all values.
+        println("Print all values.")
         c.iterator() foreach println
     }
 


[40/43] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5

Posted by yz...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: dae1b87af17abd83d072beb11243976bac943b2a
Parents: 4976361 e8a14da
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 13:25:18 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 13:25:18 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/IgniteBinary.java     | 10 +++++-----
 .../main/java/org/apache/ignite/binary/BinaryObject.java  |  2 +-
 .../main/java/org/apache/ignite/binary/BinaryType.java    |  2 +-
 .../internal/processors/cache/GridCacheMessage.java       |  1 +
 .../processors/cache/portable/IgniteBinaryImpl.java       |  9 +++++----
 .../internal/processors/cacheobject/NoOpBinary.java       |  8 ++++----
 .../portable/BinaryObjectBuilderAdditionalSelfTest.java   |  6 +++---
 .../internal/portable/GridPortableMetaDataSelfTest.java   | 10 +++++-----
 .../cache/distributed/IgniteCacheSizeFailoverTest.java    |  7 +++++++
 ...dCacheClientNodeBinaryObjectMetadataMultinodeTest.java |  8 ++++----
 .../GridCacheClientNodeBinaryObjectMetadataTest.java      |  4 ++--
 .../platform/PlatformComputeBinarizableArgTask.java       |  2 +-
 .../Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs     |  5 ++++-
 .../Apache.Ignite.Examples/Messaging/MessagingExample.cs  |  9 +++++++--
 14 files changed, 50 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[02/43] ignite git commit: Make binary marshaller default WIP.

Posted by yz...@apache.org.
Make binary marshaller default WIP.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 7ea000aa0ad4a93dac7b6e7c6e15f4c928039556
Parents: 171bbee
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 15:05:13 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 15:05:13 2015 +0300

----------------------------------------------------------------------
 .../config/binary/example-ignite-binary.xml     |  44 ------
 .../binary/ExampleBinaryNodeStartup.java        |  36 -----
 ...ComputeClientBinaryTaskExecutionExample.java |   6 +-
 .../CacheClientBinaryPutGetExample.java         |   6 +-
 .../datagrid/CacheClientBinaryQueryExample.java |   6 +-
 .../java/org/apache/ignite/IgniteBinary.java    |  11 +-
 .../java/org/apache/ignite/IgniteCache.java     |   3 +-
 .../apache/ignite/binary/BinaryIdMapper.java    |   3 +-
 .../apache/ignite/binary/BinarySerializer.java  |   2 +-
 .../configuration/CacheConfiguration.java       |  27 ++++
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../internal/portable/BinaryMarshaller.java     | 144 ++++++++++++++++++
 .../portable/PortableClassDescriptor.java       |   2 +-
 .../internal/portable/PortableContext.java      |   1 -
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../CacheObjectBinaryProcessorImpl.java         |   2 +-
 .../cache/store/CacheOsStoreManager.java        |   2 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   2 +-
 .../marshaller/portable/BinaryMarshaller.java   | 146 -------------------
 .../portable/BinaryFieldsAbstractSelfTest.java  |   1 -
 .../portable/BinaryFieldsHeapSelfTest.java      |   2 -
 .../portable/BinaryFieldsOffheapSelfTest.java   |   1 -
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   1 -
 .../BinaryFooterOffsetsHeapSelfTest.java        |   2 -
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   1 -
 .../portable/BinaryMarshallerSelfTest.java      |   1 -
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   1 -
 .../portable/BinaryObjectBuilderSelfTest.java   |   1 -
 .../GridPortableAffinityKeySelfTest.java        |   1 -
 ...idPortableMarshallerCtxDisabledSelfTest.java |   1 -
 .../portable/GridPortableMetaDataSelfTest.java  |   1 -
 .../portable/GridPortableWildcardsSelfTest.java |   1 -
 ...IgniteCacheAbstractExecutionContextTest.java |   2 +-
 ...naryObjectsAbstractDataStreamerSelfTest.java |   2 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |   2 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |   2 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   2 +-
 .../GridCachePortableStoreAbstractSelfTest.java |   2 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   2 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |   2 +-
 .../DataStreamProcessorPortableSelfTest.java    |   2 +-
 .../GridDataStreamerImplSelfTest.java           |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 .../GridCacheMemoryModePortableSelfTest.java    |   2 +-
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   2 +-
 ...eapTieredEvictionAtomicPortableSelfTest.java |   2 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |   2 +-
 .../GridCacheOffHeapTieredPortableSelfTest.java |   2 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |   2 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   2 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   2 +-
 .../IgnitePortableCacheTestSuite.java           |   2 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   2 -
 .../IgnitePortableCacheQueryTestSuite.java      |   2 +-
 .../Config/Compute/compute-grid1.xml            |   5 -
 .../Config/marshaller-explicit.xml              |   2 +-
 58 files changed, 219 insertions(+), 298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/examples/config/binary/example-ignite-binary.xml
----------------------------------------------------------------------
diff --git a/examples/config/binary/example-ignite-binary.xml b/examples/config/binary/example-ignite-binary.xml
deleted file mode 100644
index dbec5e9..0000000
--- a/examples/config/binary/example-ignite-binary.xml
+++ /dev/null
@@ -1,44 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<!--
-    Ignite configuration with all defaults and enabled p2p deployment, events and portable marshaller.
-
-    Use this configuration file when running HTTP REST examples (see 'examples/rest' folder).
-
-    When starting a standalone node, you need to execute the following command:
-    {IGNITE_HOME}/bin/ignite.{bat|sh} examples/config/portable/example-ignite-portable.xml
-
-    When starting Ignite from Java IDE, pass path to this file to Ignition:
-    Ignition.start("examples/config/portable/example-ignite-portable.xml");
--->
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd">
-    <!-- Imports default Ignite configuration -->
-    <import resource="../example-default.xml"/>
-
-    <bean parent="ignite.cfg">
-        <!-- Enables portable marshaller -->
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.portable.BinaryMarshaller"/>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/examples/src/main/java/org/apache/ignite/examples/binary/ExampleBinaryNodeStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/ExampleBinaryNodeStartup.java b/examples/src/main/java/org/apache/ignite/examples/binary/ExampleBinaryNodeStartup.java
deleted file mode 100644
index 30a015b..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/binary/ExampleBinaryNodeStartup.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.binary;
-
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.Ignition;
-
-/**
- * Starts up an empty node with example configuration and binary marshaller enabled.
- */
-public class ExampleBinaryNodeStartup {
-    /**
-     * Start up an empty node with example configuration and binary marshaller enabled.
-     *
-     * @param args Command line arguments, none required.
-     * @throws IgniteException If failed.
-     */
-    public static void main(String[] args) throws IgniteException {
-        Ignition.start("examples/config/binary/example-ignite-binary.xml");
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
index ae5d169..ba1c968 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/computegrid/ComputeClientBinaryTaskExecutionExample.java
@@ -37,8 +37,8 @@ import org.apache.ignite.binary.BinaryObject;
  * Remote nodes should always be started with special configuration file which
  * enables the binary marshaller: {@code 'ignite.{sh|bat} examples/config/binary/example-ignite-binary.xml'}.
  * <p>
- * Alternatively you can run {@link org.apache.ignite.examples.binary.ExampleBinaryNodeStartup} in another JVM which will
- * start node with {@code examples/config/binary/example-ignite-binary.xml} configuration.
+ * Alternatively you can run {@link org.apache.ignite.examples.ExampleNodeStartup} in another JVM which will
+ * start a node with {@code examples/config/example-ignite.xml} configuration.
  */
 public class ComputeClientBinaryTaskExecutionExample {
     /**
@@ -47,7 +47,7 @@ public class ComputeClientBinaryTaskExecutionExample {
      * @param args Command line arguments, none required.
      */
     public static void main(String[] args) {
-        try (Ignite ignite = Ignition.start("examples/config/binary/example-ignite-binary.xml")) {
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
             System.out.println();
             System.out.println(">>> Binary objects task execution example started.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
index bb88ee0..e49c16a 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryPutGetExample.java
@@ -44,8 +44,8 @@ import org.apache.ignite.binary.BinaryObject;
  * Remote nodes should always be started with special configuration file which
  * enables the binary marshaller: {@code 'ignite.{sh|bat} examples/config/binary/example-ignite-binary.xml'}.
  * <p>
- * Alternatively you can run {@link org.apache.ignite.examples.binary.ExampleBinaryNodeStartup} in another JVM which will
- * start node with {@code examples/config/binary/example-ignite-binary.xml} configuration.
+ * Alternatively you can run {@link org.apache.ignite.examples.ExampleNodeStartup} in another JVM which will
+ * start node with {@code examples/config/example-ignite.xml} configuration.
  */
 @SuppressWarnings("TypeMayBeWeakened")
 public class CacheClientBinaryPutGetExample {
@@ -58,7 +58,7 @@ public class CacheClientBinaryPutGetExample {
      * @param args Command line arguments, none required.
      */
     public static void main(String[] args) {
-        try (Ignite ignite = Ignition.start("examples/config/binary/example-ignite-binary.xml")) {
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
             System.out.println();
             System.out.println(">>> Binary objects cache put-get example started.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
index c93848c..ed01eb9 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/CacheClientBinaryQueryExample.java
@@ -45,8 +45,8 @@ import org.apache.ignite.binary.BinaryObject;
  * This example demonstrates use of binary objects with cache queries.
  * The example populates cache with sample data and runs several SQL and full text queries over this data.
  * <p>
- * Remote nodes should always be started with {@link org.apache.ignite.examples.binary.ExampleBinaryNodeStartup} which starts a node with
- * {@code examples/config/binary/example-ignite-binary.xml} configuration.
+ * Remote nodes should always be started with {@link org.apache.ignite.examples.ExampleNodeStartup} which starts
+ * a node with {@code examples/config/example-ignite.xml} configuration.
  */
 public class CacheClientBinaryQueryExample {
     /** Organization cache name. */
@@ -63,7 +63,7 @@ public class CacheClientBinaryQueryExample {
      * @param args Command line arguments, none required.
      */
     public static void main(String[] args) {
-        try (Ignite ignite = Ignition.start("examples/config/binary/example-ignite-binary.xml")) {
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
             System.out.println();
             System.out.println(">>> Binary objects cache query example started.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java b/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
index 5eb2670..5340266 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
@@ -67,9 +67,7 @@ import org.jetbrains.annotations.Nullable;
  * String field = val.field("myFieldName");
  * </pre>
  * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link org.apache.ignite.marshaller.portable.BinaryMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
+ * typed objects at all times. In this case we do incur the deserialization cost.
  * <pre name=code class=java>
  * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
  *
@@ -150,8 +148,6 @@ import org.jetbrains.annotations.Nullable;
  * automatically.
  * <h1 class="header">Configuration</h1>
  * By default all your objects are considered as binary and no specific configuration is needed.
- * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
- * explicitly by calling {@link org.apache.ignite.marshaller.portable.BinaryMarshaller#setClassNames(Collection)}.
  * The only requirement Ignite imposes is that your object has an empty
  * constructor. Note, that since server side does not have to know the class definition,
  * you only need to list portable objects in configuration on the client side. However, if you
@@ -235,7 +231,8 @@ import org.jetbrains.annotations.Nullable;
  * }
  * </pre>
  * Alternatively, if you cannot change class definitions, you can provide custom serialization
- * logic in {@link org.apache.ignite.binary.BinarySerializer} either globally in {@link org.apache.ignite.marshaller.portable.BinaryMarshaller} or
+ * logic in {@link org.apache.ignite.binary.BinarySerializer} either globally in
+ * {@link org.apache.ignite.configuration.BinaryConfiguration} or
  * for a specific type via {@link org.apache.ignite.binary.BinaryTypeConfiguration} instance.
  * <p>
  * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
@@ -255,7 +252,7 @@ import org.jetbrains.annotations.Nullable;
  * internally. However, in cases when you want to provide your own ID mapping schema,
  * you can provide your own {@link org.apache.ignite.binary.BinaryIdMapper} implementation.
  * <p>
- * ID-mapper may be provided either globally in {@link org.apache.ignite.marshaller.portable.BinaryMarshaller},
+ * ID-mapper may be provided either globally in {@link org.apache.ignite.configuration.BinaryConfiguration},
  * or for a specific type via {@link org.apache.ignite.binary.BinaryTypeConfiguration} instance.
  * <h1 class="header">Query Indexing</h1>
  * Portable objects can be indexed for querying by specifying index fields in

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index c9de3f1..c79bb12 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -162,7 +162,8 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * BinaryObject po = prj.get(1);
      * </pre>
      * <p>
-     * Note that this method makes sense only if cache is working in binary mode ({@link org.apache.ignite.marshaller.portable.BinaryMarshaller} is used).
+     * Note that this method makes sense only if cache is working in binary mode
+     * if default marshaller is used.
      * If not, this method is no-op and will return current cache.
      *
      * @return New cache instance for portable objects.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
index a4a18f6..4c31140 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryIdMapper.java
@@ -27,7 +27,8 @@ package org.apache.ignite.binary;
  * actually do collide {@code BinaryIdMapper} allows to override the automatically
  * generated hash code IDs for the type and field names.
  * <p>
- * Binary ID mapper can be configured for all binary objects via {@link org.apache.ignite.marshaller.portable.BinaryMarshaller#getIdMapper()} method,
+ * Binary ID mapper can be configured for all binary objects via
+ * {@link org.apache.ignite.configuration.BinaryConfiguration#getIdMapper()} method,
  * or for a specific binary type via {@link BinaryTypeConfiguration#getIdMapper()} method.
  */
 public interface BinaryIdMapper {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java b/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
index 31b3d30..3a8be3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
@@ -23,7 +23,7 @@ package org.apache.ignite.binary;
  * cannot be changed directly.
  * <p>
  * Binary serializer can be configured for all binary objects via
- * {@link org.apache.ignite.marshaller.portable.BinaryMarshaller#getSerializer()} method, or for a specific
+ * {@link org.apache.ignite.configuration.BinaryConfiguration#getSerializer()} method, or for a specific
  * binary type via {@link BinaryTypeConfiguration#getSerializer()} method.
  */
 public interface BinarySerializer {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index e87346f..e7dd9c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -45,6 +45,7 @@ import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheInterceptor;
+import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
@@ -388,6 +389,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Query entities. */
     private Collection<QueryEntity> qryEntities;
 
+    /** Cache  */
+    private Collection<CacheKeyConfiguration> cacheKeyCfgs;
+
     /** Empty constructor (all values are initialized to their defaults). */
     public CacheConfiguration() {
         /* No-op. */
@@ -422,6 +426,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         atomicityMode = cc.getAtomicityMode();
         atomicWriteOrderMode = cc.getAtomicWriteOrderMode();
         backups = cc.getBackups();
+        cacheKeyCfgs = cc.getCacheKeyConfiguration();
         cacheLoaderFactory = cc.getCacheLoaderFactory();
         cacheMode = cc.getCacheMode();
         cacheWriterFactory = cc.getCacheWriterFactory();
@@ -992,6 +997,28 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * Gets cache key configuration. Cache key configuration allows to override default cache key affinity
+     * by specifying affinity field name.
+     *
+     * @return Cache key configuration.
+     */
+    public Collection<CacheKeyConfiguration> getCacheKeyConfiguration() {
+        return cacheKeyCfgs;
+    }
+
+    /**
+     * Sets cache key configuration.
+     *
+     * @param cacheKeyCfgs Collection of cache key configurations.
+     */
+    public void setCacheKeyConfiguration(Collection<CacheKeyConfiguration> cacheKeyCfgs) {
+        if (this.cacheKeyCfgs != null)
+            this.cacheKeyCfgs.addAll(cacheKeyCfgs);
+        else
+            this.cacheKeyCfgs = cacheKeyCfgs;
+    }
+
+    /**
      * Gets cache atomicity mode.
      * <p>
      * Default value is defined by {@link #DFLT_CACHE_ATOMICITY_MODE}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 082ffa5..305769b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -158,7 +158,7 @@ import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
new file mode 100644
index 0000000..4f8c83c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
@@ -0,0 +1,144 @@
+/*
+ * 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.portable;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.marshaller.AbstractMarshaller;
+import org.apache.ignite.marshaller.MarshallerContext;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
+ * in the portable format.
+ * <p>
+ * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This marshaller has no mandatory configuration parameters.
+ * <h2 class="header">Java Example</h2>
+ * <pre name="code" class="java">
+ * PortableMarshaller marshaller = new PortableMarshaller();
+ *
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * // Override marshaller.
+ * cfg.setMarshaller(marshaller);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * PortableMarshaller can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
+ *     ...
+ *     &lt;property name="marshaller"&gt;
+ *         &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
+ *            ...
+ *         &lt;/bean&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&gt;
+ * </pre>
+ * <p>
+ * <img src="http://ignite.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ */
+public class BinaryMarshaller extends AbstractMarshaller {
+    /** */
+    private GridPortableMarshaller impl;
+
+    /**
+     * Returns currently set {@link MarshallerContext}.
+     *
+     * @return Marshaller context.
+     */
+    public MarshallerContext getContext() {
+        return ctx;
+    }
+
+    /**
+     * Sets {@link PortableContext}.
+     * <p/>
+     * @param ctx Portable context.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private void setPortableContext(PortableContext ctx, IgniteConfiguration cfg) {
+        ctx.configure(this, cfg);
+
+        impl = new GridPortableMarshaller(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
+        return impl.marshal(obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
+        byte[] arr = marshal(obj);
+
+        try {
+            out.write(arr);
+        }
+        catch (IOException e) {
+            throw new BinaryObjectException("Failed to marshal the object: " + obj, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        return impl.deserialize(bytes, clsLdr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+
+        // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that
+        // returns number of bytes remaining.
+        try {
+            byte[] arr = new byte[4096];
+
+            int cnt;
+
+            while ((cnt = in.read(arr)) != -1)
+                buf.write(arr, 0, cnt);
+
+            buf.flush();
+
+            return impl.deserialize(buf.toByteArray(), clsLdr);
+        }
+        catch (IOException e) {
+            throw new BinaryObjectException("Failed to unmarshal the object from InputStream", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onUndeploy(ClassLoader ldr) {
+        impl.context().onUndeploy(ldr);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 974f891..d3c002e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -328,7 +328,7 @@ public class PortableClassDescriptor {
     }
 
     /**
-     * @return {@code true} if {@link OptimizedMarshaller} must be used instead of {@link org.apache.ignite.marshaller.portable.BinaryMarshaller}
+     * @return {@code true} if {@link OptimizedMarshaller} must be used instead of {@link BinaryMarshaller}
      * for object serialization and deserialization.
      */
     public boolean useOptimizedMarshaller() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 01bc9d8..8f37517 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -71,7 +71,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 4439eee..178c5f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -112,7 +112,7 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.jetbrains.annotations.Nullable;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 05e9263..483b6ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -69,7 +69,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
index 29d5155..385b4b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
@@ -23,7 +23,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  * Default store manager implementation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
index b53cc46..4bd311b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cpp/PlatformCppConfigurationClosure.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurat
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.platform.cpp.PlatformCppConfiguration;
 
 import java.util.Collections;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/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 a0c5a0b..c506a35 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
@@ -38,7 +38,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/main/java/org/apache/ignite/marshaller/portable/BinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/BinaryMarshaller.java
deleted file mode 100644
index 455c83e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/BinaryMarshaller.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.marshaller.portable;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableContext;
-import org.apache.ignite.marshaller.AbstractMarshaller;
-import org.apache.ignite.marshaller.MarshallerContext;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
- * in the portable format.
- * <p>
- * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
- * <p>
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This marshaller has no mandatory configuration parameters.
- * <h2 class="header">Java Example</h2>
- * <pre name="code" class="java">
- * PortableMarshaller marshaller = new PortableMarshaller();
- *
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * // Override marshaller.
- * cfg.setMarshaller(marshaller);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * PortableMarshaller can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
- *     ...
- *     &lt;property name="marshaller"&gt;
- *         &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *            ...
- *         &lt;/bean&gt;
- *     &lt;/property&gt;
- *     ...
- * &lt;/bean&gt;
- * </pre>
- * <p>
- * <img src="http://ignite.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- */
-public class BinaryMarshaller extends AbstractMarshaller {
-    /** */
-    private GridPortableMarshaller impl;
-
-    /**
-     * Returns currently set {@link MarshallerContext}.
-     *
-     * @return Marshaller context.
-     */
-    public MarshallerContext getContext() {
-        return ctx;
-    }
-
-    /**
-     * Sets {@link PortableContext}.
-     * <p/>
-     * @param ctx Portable context.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private void setPortableContext(PortableContext ctx, IgniteConfiguration cfg) {
-        ctx.configure(this, cfg);
-
-        impl = new GridPortableMarshaller(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
-        return impl.marshal(obj);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
-        byte[] arr = marshal(obj);
-
-        try {
-            out.write(arr);
-        }
-        catch (IOException e) {
-            throw new BinaryObjectException("Failed to marshal the object: " + obj, e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        return impl.deserialize(bytes, clsLdr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-
-        // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that
-        // returns number of bytes remaining.
-        try {
-            byte[] arr = new byte[4096];
-
-            int cnt;
-
-            while ((cnt = in.read(arr)) != -1)
-                buf.write(arr, 0, cnt);
-
-            buf.flush();
-
-            return impl.deserialize(buf.toByteArray(), clsLdr);
-        }
-        catch (IOException e) {
-            throw new BinaryObjectException("Failed to unmarshal the object from InputStream", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onUndeploy(ClassLoader ldr) {
-        impl.context().onUndeploy(ldr);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index dd08390..b651db8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
@@ -25,7 +25,6 @@ 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.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.math.BigDecimal;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
index 53ba212..d7192e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
-
 /**
  * Field tests for heap-based portables.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
index 68bfb30..e353596 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.eclipse.jetty.util.ConcurrentHashSet;
 import sun.misc.Unsafe;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
index 7267b16..036bf8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
index 471bd44..c71cb5e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
-
 /**
  * Compact offsets tests for heap portable objects.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
index 7b44b80..536b8b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.eclipse.jetty.util.ConcurrentHashSet;
 import sun.misc.Unsafe;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
index 4584575..34bd4a1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
@@ -41,7 +41,6 @@ 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.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jsr166.ConcurrentHashMap8;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
index 356a25b..31ecd36 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
@@ -46,7 +46,6 @@ import org.apache.ignite.internal.portable.mutabletest.GridBinaryMarshalerAwareT
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.portable.IgniteBinaryImpl;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryObject;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
index 7f023f3..a57ed0d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
@@ -44,7 +44,6 @@ import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.T
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import sun.misc.Unsafe;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
index 9fb3a6f..3851bbc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
@@ -33,7 +33,6 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteRunnable;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index 917298a..346cf80 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -25,7 +25,6 @@ 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.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.io.Externalizable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
index 5c25d3b..c3521c4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
@@ -26,7 +26,6 @@ import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.binary.BinaryType;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
index a00ad75..0fb5381 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
@@ -27,7 +27,6 @@ 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.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
index c855c9d..cc4d228 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.GridTestExternalClassLoader;
 import org.apache.ignite.testframework.config.GridTestProperties;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractDataStreamerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractDataStreamerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractDataStreamerSelfTest.java
index 7ee6cb6..3938bc6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractDataStreamerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractDataStreamerSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.binary.BinaryReader;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractMultiThreadedSelfTest.java
index 5fb02b6..f1e427b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractMultiThreadedSelfTest.java
@@ -33,7 +33,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.binary.BinaryObject;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java
index 3925045..29b7204 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -46,7 +46,7 @@ import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.Binarylizable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
index 99e2073..af38b74 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
@@ -33,7 +33,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
index 256e5fd..9d296cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
@@ -30,7 +30,7 @@ import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
index d3df9b7..892a891 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStoreAbstractSelfTest.java
@@ -28,7 +28,7 @@ import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
index 30a7ca5..a965588 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
@@ -25,7 +25,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfT
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
index b01a363..d15f321 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObject;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
index 3777a20..0599863 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.stream.StreamReceiver;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
index 6ce5961..72f71b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
@@ -30,7 +30,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.binary.BinaryObject;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
index c18260b..a6ff3ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
@@ -22,7 +22,7 @@ import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityRoutingSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
index 0f3e67c..71cf106 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheMemoryModePortableSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheMemoryModeSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  * Memory models test.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java
index 8996355..80353c7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredAtomicPortableSelfTest.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredAtomicSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
index 1eb5d1a..a8c87ea 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionAtomicSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObject;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
index 85f0298..0665f20 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.binary.BinaryObject;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java
index 22c9a70..8ab7f42 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredPortableSelfTest.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
index 8e4bb35..ce3a9e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesNearPartitionedByteArrayValuesSelfTest.java
@@ -18,7 +18,7 @@ package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAbstractNearPartitionedByteArrayValuesSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
index 256e0c2..e20166e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
index 5c5bb5a..f0f5fcc 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheFullApiTestSuite.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
index 09a0adb..6e34a98 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheTestSuite.java
@@ -44,7 +44,7 @@ import org.apache.ignite.internal.processors.cache.portable.distributed.dht.Grid
 import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortablesNearPartitionedByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortablesPartitionedOnlyByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/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 4d9456a..d967e3b 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
@@ -24,10 +24,8 @@ import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
 
 /**
  * Checks behavior on exception while unmarshalling key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ea000aa/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
index b241b86..4e9be93 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
@@ -45,7 +45,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 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;
-import org.apache.ignite.marshaller.portable.BinaryMarshaller;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
 import org.apache.ignite.testframework.config.GridTestProperties;
 


[24/43] ignite git commit: Fixed test to do not fail with non-full-sync mode.

Posted by yz...@apache.org.
Fixed test to do not fail with non-full-sync mode.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 041cd38d46b458ea3ae945e15411d4519c1bf777
Parents: 28d6e00
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 10:09:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 10:09:46 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteClientReconnectCacheTest.java       | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/041cd38d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
index 6cf10f4..3d8f601 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
@@ -56,6 +56,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLock
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
@@ -1292,6 +1293,12 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
 
         cache.put(1, 1);
 
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return cache.get(1) != null;
+            }
+        }, 5000);
+
         assertEquals(1, cache.get(1));
     }
 


[13/43] ignite git commit: Merge branch 'ignite-1282' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282

Posted by yz...@apache.org.
Merge branch 'ignite-1282' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 76803923dda7fb346e273aca5079352532b586dd
Parents: 4844b3e b751843
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 14:41:22 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 14:41:22 2015 +0300

----------------------------------------------------------------------
 .../scalar/examples/ScalarCacheExample.scala     | 19 +++++++++++++++----
 .../test/resources/spring-ping-pong-partner.xml  | 18 ------------------
 2 files changed, 15 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[35/43] ignite git commit: Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.5' into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 446a14bf67f81d01e439867e5a3783cab4d59275
Parents: 0a97c1c f2092e7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 26 12:42:33 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 26 12:42:33 2015 +0300

----------------------------------------------------------------------
 .../config/binary/example-ignite-binary.xml     |  44 ------
 examples/config/example-ignite.xml              |  10 +-
 .../ignite/examples/binary/EmployeeKey.java     |   3 +
 .../binary/ExampleBinaryNodeStartup.java        |  36 -----
 ...ComputeClientBinaryTaskExecutionExample.java |   6 +-
 .../CacheClientBinaryPutGetExample.java         |   6 +-
 .../datagrid/CacheClientBinaryQueryExample.java |   6 +-
 .../scalar/examples/ScalarCacheExample.scala    |  19 ++-
 .../ignite/codegen/MessageCodeGenerator.java    |   1 +
 .../java/org/apache/ignite/IgniteBinary.java    |   5 +-
 .../java/org/apache/ignite/IgniteCache.java     |   3 +-
 .../apache/ignite/binary/BinaryIdMapper.java    |   3 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |   2 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |   2 +-
 .../configuration/CacheConfiguration.java       |   6 +-
 .../internal/GridEventConsumeHandler.java       |   5 +
 .../internal/GridMessageListenHandler.java      |   5 +
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   6 +-
 .../ignite/internal/NodeStoppingException.java  |  35 +++++
 .../internal/portable/BinaryMarshaller.java     | 144 ++++++++++++++++++
 .../portable/PortableClassDescriptor.java       |   2 +-
 .../internal/portable/PortableContext.java      |  35 ++++-
 .../builder/BinaryObjectBuilderImpl.java        |   9 +-
 .../processors/cache/CacheObjectContext.java    |   2 +-
 .../cache/GridCacheDeploymentManager.java       |   6 +-
 .../processors/cache/GridCacheEventManager.java |  38 ++++-
 .../processors/cache/GridCacheIoManager.java    |   6 +
 .../processors/cache/GridCacheMessage.java      |   3 +-
 .../processors/cache/GridCacheMvccManager.java  |   3 +-
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../processors/cache/IgniteCacheProxy.java      |  14 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |  19 +--
 .../distributed/dht/GridDhtTxFinishFuture.java  |   1 +
 .../dht/GridPartitionedSingleGetFuture.java     |   3 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   2 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   3 +-
 .../CacheObjectBinaryProcessorImpl.java         |  30 +++-
 .../continuous/CacheContinuousQueryEntry.java   |  50 +++++--
 .../continuous/CacheContinuousQueryEvent.java   |   6 +-
 .../continuous/CacheContinuousQueryHandler.java |  32 +++-
 .../CacheContinuousQueryListener.java           |   5 +
 .../continuous/CacheContinuousQueryManager.java |  57 +++++---
 .../cache/store/CacheOsStoreManager.java        |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  29 +++-
 .../continuous/GridContinuousHandler.java       |   5 +
 .../continuous/GridContinuousProcessor.java     |  10 +-
 .../StartRoutineDiscoveryMessage.java           |  13 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   2 +-
 .../processors/query/GridQueryProcessor.java    |  28 +++-
 .../marshaller/portable/BinaryMarshaller.java   | 146 -------------------
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |  85 +++++++++++
 ...JdbcPojoStorePortableMarshallerSelfTest.java |  85 -----------
 .../portable/BinaryFieldsAbstractSelfTest.java  |   1 -
 .../portable/BinaryFieldsHeapSelfTest.java      |   2 -
 .../portable/BinaryFieldsOffheapSelfTest.java   |   1 -
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   1 -
 .../BinaryFooterOffsetsHeapSelfTest.java        |   2 -
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   1 -
 .../portable/BinaryMarshallerSelfTest.java      |   1 -
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   1 -
 .../portable/BinaryObjectBuilderSelfTest.java   |  31 +++-
 .../GridPortableAffinityKeySelfTest.java        |  13 +-
 ...idPortableMarshallerCtxDisabledSelfTest.java |   1 -
 .../portable/GridPortableMetaDataSelfTest.java  |   1 -
 .../portable/GridPortableWildcardsSelfTest.java |   1 -
 .../cache/GridCacheAbstractFullApiSelfTest.java |  97 ++++++++++++
 ...IgniteCacheAbstractExecutionContextTest.java |   2 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |   2 +
 ...naryObjectsAbstractDataStreamerSelfTest.java |   2 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |   2 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |   2 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   2 +-
 .../GridCachePortableStoreAbstractSelfTest.java |   2 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   2 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |   2 +-
 .../DataStreamProcessorPortableSelfTest.java    |   2 +-
 .../GridDataStreamerImplSelfTest.java           |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 .../GridCacheMemoryModePortableSelfTest.java    |   2 +-
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   2 +-
 ...eapTieredEvictionAtomicPortableSelfTest.java |   2 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |   2 +-
 .../GridCacheOffHeapTieredPortableSelfTest.java |   2 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |   2 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   2 +-
 .../ignite/testframework/junits/IgniteMock.java |  39 ++++-
 .../IgniteBinaryObjectsCacheTestSuite2.java     |  33 +++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   2 +-
 .../IgnitePortableCacheTestSuite.java           |   2 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  23 +++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   2 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   2 -
 .../IgnitePortableCacheQueryTestSuite.java      |   2 +-
 .../Config/Compute/compute-grid1.xml            |   5 -
 .../Config/marshaller-explicit.xml              |   2 +-
 .../test/resources/spring-ping-pong-partner.xml |  18 ---
 101 files changed, 910 insertions(+), 514 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/446a14bf/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/446a14bf/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/446a14bf/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/446a14bf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/446a14bf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
----------------------------------------------------------------------


[25/43] ignite git commit: Disabled events in test, otherwise get OOM when run it with -Xmx2g.

Posted by yz...@apache.org.
Disabled events in test, otherwise get OOM when run it with -Xmx2g.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: d99fc8d12e9dd022eaa9664437fe1ff9b19d6863
Parents: 041cd38
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 11:12:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 11:12:48 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d99fc8d1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index ee28cf9..6bbca88 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -110,6 +110,8 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setIncludeEventTypes();
+
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
 
         ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);


[16/43] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282

Posted by yz...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 0f1dd6f53a203b3530ca98db37405eac06b18c81
Parents: 8e1d6c0 dafad52
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Nov 25 09:46:10 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Nov 25 09:46:10 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  9 +++--
 .../processors/cache/CacheLockImpl.java         | 18 ++++++++-
 .../colocated/GridDhtColocatedLockFuture.java   |  8 ++--
 .../cache/GridCacheAbstractFullApiSelfTest.java | 42 +++++++++++++++++++-
 5 files changed, 67 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0f1dd6f5/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------


[21/43] ignite git commit: Fixing marshalToPortable and IgniteBiTuple

Posted by yz...@apache.org.
Fixing marshalToPortable and IgniteBiTuple


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: afcf0ab5e5d8e33351cf37790894c1d71ac6cc65
Parents: 809a442
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 09:21:50 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 09:21:50 2015 +0300

----------------------------------------------------------------------
 .../cache/portable/CacheObjectBinaryProcessorImpl.java   | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/afcf0ab5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index ba7d6f8..819b57e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -62,11 +62,13 @@ import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.internal.portable.BinaryMarshaller;
@@ -390,6 +392,15 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             return pArr;
         }
 
+        if (obj instanceof IgniteBiTuple) {
+            IgniteBiTuple tup = (IgniteBiTuple)obj;
+
+            if (obj instanceof T2)
+                return new T2<>(marshalToPortable(tup.get1()), marshalToPortable(tup.get2()));
+
+            return new IgniteBiTuple<>(marshalToPortable(tup.get1()), marshalToPortable(tup.get2()));
+        }
+
         if (obj instanceof Collection) {
             Collection<Object> col = (Collection<Object>)obj;
 


[28/43] ignite git commit: Added cache2 suite for binary objects.

Posted by yz...@apache.org.
Added cache2 suite for binary objects.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 30f01f52a60d65c930812715ec8d8b371cfa93b5
Parents: 7eea46d
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 11:54:39 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 11:54:39 2015 +0300

----------------------------------------------------------------------
 .../IgniteBinaryObjectsCacheTestSuite2.java     | 33 ++++++++++++++++++++
 1 file changed, 33 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/30f01f52/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite2.java
new file mode 100644
index 0000000..e8064e7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite2.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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ *
+ */
+public class IgniteBinaryObjectsCacheTestSuite2 {
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
+        return IgniteCacheTestSuite2.suite();
+    }
+}


[07/43] ignite git commit: IGNITE-1282 Fixed config for scalar examples.

Posted by yz...@apache.org.
IGNITE-1282 Fixed config for scalar examples.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 1e732a0f9f38257e7d74960f7b9fe6caa4fcce30
Parents: 01c24e7
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Nov 24 13:52:18 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 24 13:52:18 2015 +0700

----------------------------------------------------------------------
 .../test/resources/spring-ping-pong-partner.xml   | 18 ------------------
 1 file changed, 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1e732a0f/modules/scalar/src/test/resources/spring-ping-pong-partner.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/src/test/resources/spring-ping-pong-partner.xml b/modules/scalar/src/test/resources/spring-ping-pong-partner.xml
index adc39ec..0f35c44 100644
--- a/modules/scalar/src/test/resources/spring-ping-pong-partner.xml
+++ b/modules/scalar/src/test/resources/spring-ping-pong-partner.xml
@@ -42,24 +42,6 @@
         <property name="peerClassLoadingEnabled" value="true"/>
 
         <!--
-            Configure optimized marshaller.
-        -->
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
-                <!--
-                    For better performance set this property to true in case
-                    all marshalled classes implement java.io.Serializable.
-                    Default value is true.
-
-                    Note, that it is recommended to implement java.io.Externalizable
-                    instead of java.io.Serializable for smaller network footprint
-                    and even better performance.
-                -->
-                <property name="requireSerializable" value="false"/>
-            </bean>
-        </property>
-
-        <!--
             Enable cache events.
         -->
         <property name="includeEventTypes">


[23/43] ignite git commit: Disabled shmem communication in failover tests.

Posted by yz...@apache.org.
Disabled shmem communication in failover tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 28d6e006c40b83fc8744b44312e74779264c0265
Parents: 4af461a
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 09:55:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 09:55:24 2015 +0300

----------------------------------------------------------------------
 .../GridCacheAbstractDataStructuresFailoverSelfTest.java          | 3 +++
 .../cache/datastructures/IgniteCollectionAbstractTest.java        | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/28d6e006/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
index bc11448..4ee200b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
@@ -54,6 +54,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
 
@@ -122,6 +123,8 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         AtomicConfiguration atomicCfg = new AtomicConfiguration();
 
         atomicCfg.setCacheMode(collectionCacheMode());

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d6e006/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java
index 36c846a..3e38a58 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.datastructures.GridCacheQueueAdapter;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -45,6 +46,8 @@ public abstract class IgniteCollectionAbstractTest extends GridCommonAbstractTes
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         TcpDiscoverySpi spi = new TcpDiscoverySpi();
 
         spi.setIpFinder(ipFinder);


[22/43] ignite git commit: Queries optimizations: - cache prepared statements and two-step queries - skip reduce query if possible - do not use index snapshots by default

Posted by yz...@apache.org.
Queries optimizations:
- cache prepared statements and two-step queries
- skip reduce query if possible
 - do not use index snapshots by default


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 4af461a7f403fc0e4df13d23d0789eee49200b26
Parents: 3c42da8
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 09:30:10 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 09:30:10 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  30 +++
 .../cache/query/GridCacheSqlQuery.java          |  42 ++++
 .../cache/query/GridCacheTwoStepQuery.java      |  35 ++-
 .../processors/query/GridQueryProcessor.java    |  40 +---
 .../processors/query/GridQueryProperty.java     |  45 ++++
 .../query/GridQueryTypeDescriptor.java          |   6 +
 .../query/h2/GridH2ResultSetIterator.java       |  19 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 227 ++++++++++++++++---
 .../query/h2/opt/GridH2RowDescriptor.java       |   5 +
 .../processors/query/h2/opt/GridH2Table.java    |  69 +++++-
 .../query/h2/opt/GridH2TreeIndex.java           |  98 +++++---
 .../query/h2/sql/GridSqlQuerySplitter.java      |  73 ++++--
 .../processors/query/h2/sql/GridSqlSelect.java  |  27 +++
 .../query/h2/twostep/GridMapQueryExecutor.java  |  17 +-
 .../query/h2/twostep/GridMergeIndex.java        |   7 +
 .../h2/twostep/GridMergeIndexUnsorted.java      |  14 ++
 .../h2/twostep/GridReduceQueryExecutor.java     | 117 +++++++---
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  69 +++---
 ...PartitionedSnapshotEnabledQuerySelfTest.java |  28 +++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  19 ++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 21 files changed, 776 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index e87346f..e2bf912 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -373,6 +373,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** */
     private int sqlOnheapRowCacheSize = DFLT_SQL_ONHEAP_ROW_CACHE_SIZE;
 
+    /** */
+    private boolean snapshotableIdx;
+
     /** Copy on read flag. */
     private boolean cpOnRead = DFLT_COPY_ON_READ;
 
@@ -463,6 +466,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         rebalancePoolSize = cc.getRebalanceThreadPoolSize();
         rebalanceTimeout = cc.getRebalanceTimeout();
         rebalanceThrottle = cc.getRebalanceThrottle();
+        snapshotableIdx = cc.isSnapshotableIndex();
         sqlEscapeAll = cc.isSqlEscapeAll();
         sqlFuncCls = cc.getSqlFunctionClasses();
         sqlOnheapRowCacheSize = cc.getSqlOnheapRowCacheSize();
@@ -1900,6 +1904,32 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * Gets flag indicating whether SQL indexes should support snapshots.
+     *
+     * @return {@code True} if SQL indexes should support snapshots.
+     */
+    public boolean isSnapshotableIndex() {
+        return snapshotableIdx;
+    }
+
+    /**
+     * Sets flag indicating whether SQL indexes should support snapshots.
+     * <p>
+     * Default value is {@code false}.
+     * <p>
+     * <b>Note</b> that this flag is ignored if indexes are stored in offheap memory,
+     * for offheap indexes snapshots are always enabled.
+     *
+     * @param snapshotableIdx {@code True} if SQL indexes should support snapshots.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K, V> setSnapshotableIndex(boolean snapshotableIdx) {
+        this.snapshotableIdx = snapshotableIdx;
+
+        return this;
+    }
+
+    /**
      * Gets array of cache plugin configurations.
      *
      * @return Cache plugin configurations.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index be7bbe9..e56e445 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@ -55,6 +55,16 @@ public class GridCacheSqlQuery implements Message {
     /** */
     @GridToStringInclude
     @GridDirectTransient
+    private int[] paramIdxs;
+
+    /** */
+    @GridToStringInclude
+    @GridDirectTransient
+    private int paramsSize;
+
+    /** */
+    @GridToStringInclude
+    @GridDirectTransient
     private LinkedHashMap<String, ?> cols;
 
     /** Field kept for backward compatibility. */
@@ -77,6 +87,14 @@ public class GridCacheSqlQuery implements Message {
         this.qry = qry;
 
         this.params = F.isEmpty(params) ? EMPTY_PARAMS : params;
+        paramsSize = this.params.length;
+    }
+
+    /**
+     * @param paramIdxs Parameter indexes.
+     */
+    public void parameterIndexes(int[] paramIdxs) {
+        this.paramIdxs = paramIdxs;
     }
 
     /**
@@ -222,4 +240,28 @@ public class GridCacheSqlQuery implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
+
+    /**
+     * @param args Arguments.
+     * @return Copy.
+     */
+    public GridCacheSqlQuery copy(Object[] args) {
+        GridCacheSqlQuery cp = new GridCacheSqlQuery();
+
+        cp.qry = qry;
+        cp.cols = cols;
+        cp.paramIdxs = paramIdxs;
+        cp.paramsSize = paramsSize;
+
+        if (F.isEmpty(args))
+            cp.params = EMPTY_PARAMS;
+        else {
+            cp.params = new Object[paramsSize];
+
+            for (int paramIdx : paramIdxs)
+                cp.params[paramIdx] = args[paramIdx];
+        }
+
+        return cp;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 4b5fe22..da59c18 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -47,16 +47,27 @@ public class GridCacheTwoStepQuery {
     /** */
     private Set<String> spaces;
 
+    /** */
+    private final boolean skipMergeTbl;
+
     /**
      * @param spaces All spaces accessed in query.
      * @param rdc Reduce query.
+     * @param skipMergeTbl {@code True} if reduce query can skip merge table creation and
+     *      get data directly from merge index.
      */
-    public GridCacheTwoStepQuery(Set<String> spaces, GridCacheSqlQuery rdc) {
+    public GridCacheTwoStepQuery(Set<String> spaces, GridCacheSqlQuery rdc, boolean skipMergeTbl) {
         assert rdc != null;
 
         this.spaces = spaces;
-
         this.rdc = rdc;
+        this.skipMergeTbl = skipMergeTbl;
+    }
+    /**
+     * @return {@code True} if reduce query can skip merge table creation and get data directly from merge index.
+     */
+    public boolean skipMergeTable() {
+        return skipMergeTbl;
     }
 
     /**
@@ -89,9 +100,12 @@ public class GridCacheTwoStepQuery {
 
     /**
      * @param qry SQL Query.
+     * @return {@code this}.
      */
-    public void addMapQuery(GridCacheSqlQuery qry) {
+    public GridCacheTwoStepQuery addMapQuery(GridCacheSqlQuery qry) {
         mapQrys.add(qry);
+
+        return this;
     }
 
     /**
@@ -122,6 +136,21 @@ public class GridCacheTwoStepQuery {
         this.spaces = spaces;
     }
 
+    /**
+     * @param args New arguments to copy with.
+     * @return Copy.
+     */
+    public GridCacheTwoStepQuery copy(Object[] args) {
+        assert !explain;
+
+        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(spaces, rdc.copy(args), skipMergeTbl);
+        cp.pageSize = pageSize;
+        for (int i = 0; i < mapQrys.size(); i++)
+            cp.mapQrys.add(mapQrys.get(i).copy(args));
+
+        return cp;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheTwoStepQuery.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/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 d0eeeb1..814229c 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
@@ -1672,34 +1672,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     *
-     */
-    private abstract static class Property {
-        /**
-         * Gets this property value from the given object.
-         *
-         * @param key Key.
-         * @param val Value.
-         * @return Property value.
-         * @throws IgniteCheckedException If failed.
-         */
-        public abstract Object value(Object key, Object val) throws IgniteCheckedException;
-
-        /**
-         * @return Property name.
-         */
-        public abstract String name();
-
-        /**
-         * @return Class member type.
-         */
-        public abstract Class<?> type();
-    }
-
-    /**
      * Description of type property.
      */
-    private static class ClassProperty extends Property {
+    private static class ClassProperty extends GridQueryProperty {
         /** */
         private final Member member;
 
@@ -1794,7 +1769,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    private class PortableProperty extends Property {
+    private class PortableProperty extends GridQueryProperty {
         /** Property name. */
         private String propName;
 
@@ -1938,7 +1913,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         /** */
         @GridToStringExclude
-        private final Map<String, Property> props = new HashMap<>();
+        private final Map<String, GridQueryProperty> props = new HashMap<>();
 
         /** */
         @GridToStringInclude
@@ -1993,11 +1968,16 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @Override public GridQueryProperty property(String name) {
+            return props.get(name);
+        }
+
+        /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
             assert field != null;
 
-            Property prop = props.get(field);
+            GridQueryProperty prop = props.get(field);
 
             if (prop == null)
                 throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
@@ -2096,7 +2076,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @param failOnDuplicate Fail on duplicate flag.
          * @throws IgniteCheckedException In case of error.
          */
-        public void addProperty(Property prop, boolean failOnDuplicate) throws IgniteCheckedException {
+        public void addProperty(GridQueryProperty prop, boolean failOnDuplicate) throws IgniteCheckedException {
             String name = prop.name();
 
             if (props.put(name, prop) != null && failOnDuplicate)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
new file mode 100644
index 0000000..d623d25
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Description and access method for query entity field.
+ */
+public abstract class GridQueryProperty {
+    /**
+     * Gets this property value from the given object.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @return Property value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract Object value(Object key, Object val) throws IgniteCheckedException;
+
+    /**
+     * @return Property name.
+     */
+    public abstract String name();
+
+    /**
+     * @return Class member type.
+     */
+    public abstract Class<?> type();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index b05e1d8..45919ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -50,6 +50,12 @@ public interface GridQueryTypeDescriptor {
     public <T> T value(String field, Object key, Object val) throws IgniteCheckedException;
 
     /**
+     * @param name Property name.
+     * @return Property.
+     */
+    public GridQueryProperty property(String name);
+
+    /**
      * Gets indexes for this type.
      *
      * @return Indexes for this type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
index 2c67638..3603bb5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java
@@ -41,14 +41,19 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
     protected final Object[] row;
 
     /** */
+    private final boolean closeStmt;
+
+    /** */
     private boolean hasRow;
 
     /**
      * @param data Data array.
+     * @param closeStmt If {@code true} closes result set statement when iterator is closed.
      * @throws IgniteCheckedException If failed.
      */
-    protected GridH2ResultSetIterator(ResultSet data) throws IgniteCheckedException {
+    protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt) throws IgniteCheckedException {
         this.data = data;
+        this.closeStmt = closeStmt;
 
         if (data != null) {
             try {
@@ -115,11 +120,13 @@ public abstract class GridH2ResultSetIterator<T> extends GridCloseableIteratorAd
             // Nothing to close.
             return;
 
-        try {
-            U.closeQuiet(data.getStatement());
-        }
-        catch (SQLException e) {
-            throw new IgniteCheckedException(e);
+        if (closeStmt) {
+            try {
+                U.closeQuiet(data.getStatement());
+            }
+            catch (SQLException e) {
+                throw new IgniteCheckedException(e);
+            }
         }
 
         U.closeQuiet(data);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index d5efebf..cc452c1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -79,6 +79,7 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryIndexing;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOffheap;
@@ -93,12 +94,14 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridLuceneIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
@@ -176,6 +179,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         ";DEFAULT_LOCK_TIMEOUT=10000;FUNCTIONS_IN_SCHEMA=true;OPTIMIZE_REUSE_RESULTS=0;QUERY_CACHE_SIZE=0;" +
         "RECOMPILE_ALWAYS=1;MAX_OPERATION_MEMORY=0";
 
+    /** */
+    private static final int PREPARED_STMT_CACHE_SIZE = 256;
+
+    /** */
+    private static final int TWO_STEP_QRY_CACHE_SIZE = 1024;
+
     /** Field name for key. */
     public static final String KEY_FIELD_NAME = "_key";
 
@@ -190,6 +199,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      */
     static {
         try {
+            System.setProperty("h2.objectCache", "false");
+
             COMMAND_FIELD = JdbcPreparedStatement.class.getDeclaredField("command");
 
             COMMAND_FIELD.setAccessible(true);
@@ -242,6 +253,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 c = initialValue();
 
                 set(c);
+
+                // Reset statement cache when new connection is created.
+                stmtCache.get().clear();
             }
 
             return c;
@@ -266,6 +280,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** */
     private volatile GridKernalContext ctx;
 
+    /** */
+    private final ThreadLocal<StatementCache> stmtCache = new ThreadLocal<StatementCache>() {
+        @Override protected StatementCache initialValue() {
+            return new StatementCache(PREPARED_STMT_CACHE_SIZE);
+        }
+    };
+
+    /** */
+    private final GridBoundedConcurrentLinkedHashMap<T3<String, String, Boolean>, TwoStepCachedQuery> twoStepCache =
+        new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
+
     /**
      * @param space Space.
      * @return Connection.
@@ -280,6 +305,35 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * @param c Connection.
+     * @param sql SQL.
+     * @param useStmtCache If {@code true} uses statement cache.
+     * @return Prepared statement.
+     * @throws SQLException If failed.
+     */
+    private PreparedStatement prepareStatement(Connection c, String sql, boolean useStmtCache) throws SQLException {
+        if (useStmtCache) {
+            StatementCache cache = stmtCache.get();
+
+            PreparedStatement stmt = cache.get(sql);
+
+            if (stmt != null && !stmt.isClosed()) {
+                assert stmt.getConnection() == c;
+
+                return stmt;
+            }
+
+            stmt = c.prepareStatement(sql);
+
+            cache.put(sql, stmt);
+
+            return stmt;
+        }
+        else
+            return c.prepareStatement(sql);
+    }
+
+    /**
      * Gets DB connection.
      *
      * @param schema Whether to set schema for connection or not.
@@ -648,7 +702,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             Connection conn = connectionForThread(schema(spaceName));
 
-            ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, qry, params);
+            ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, qry, params, true);
 
             List<GridQueryFieldMetadata> meta = null;
 
@@ -710,15 +764,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param conn Connection,.
      * @param sql Sql query.
      * @param params Parameters.
+     * @param useStmtCache If {@code true} uses statement cache.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    private ResultSet executeSqlQuery(Connection conn, String sql, Collection<Object> params)
+    private ResultSet executeSqlQuery(Connection conn, String sql, Collection<Object> params, boolean useStmtCache)
         throws IgniteCheckedException {
         PreparedStatement stmt;
 
         try {
-            stmt = conn.prepareStatement(sql);
+            stmt = prepareStatement(conn, sql, useStmtCache);
         }
         catch (SQLException e) {
             throw new IgniteCheckedException("Failed to parse SQL query: " + sql, e);
@@ -747,18 +802,23 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Executes sql query and prints warning if query is too slow..
      *
+     * @param space Space name.
      * @param conn Connection,.
      * @param sql Sql query.
      * @param params Parameters.
+     * @param useStmtCache If {@code true} uses statement cache.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public ResultSet executeSqlQueryWithTimer(String space, Connection conn, String sql,
-        @Nullable Collection<Object> params) throws IgniteCheckedException {
+    public ResultSet executeSqlQueryWithTimer(String space,
+        Connection conn,
+        String sql,
+        @Nullable Collection<Object> params,
+        boolean useStmtCache) throws IgniteCheckedException {
         long start = U.currentTimeMillis();
 
         try {
-            ResultSet rs = executeSqlQuery(conn, sql, params);
+            ResultSet rs = executeSqlQuery(conn, sql, params, useStmtCache);
 
             long time = U.currentTimeMillis() - start;
 
@@ -767,7 +827,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             if (time > longQryExecTimeout) {
                 String msg = "Query execution is too long (" + time + " ms): " + sql;
 
-                ResultSet plan = executeSqlQuery(conn, "EXPLAIN " + sql, params);
+                ResultSet plan = executeSqlQuery(conn, "EXPLAIN " + sql, params, false);
 
                 plan.next();
 
@@ -803,7 +863,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         String sql = generateQuery(qry, tbl);
 
-        return executeSqlQueryWithTimer(space, conn, sql, params);
+        return executeSqlQueryWithTimer(space, conn, sql, params, true);
     }
 
     /**
@@ -924,41 +984,52 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
-        String space = cctx.name();
-        String sqlQry = qry.getSql();
+        final String space = cctx.name();
+        final String sqlQry = qry.getSql();
 
         Connection c = connectionForSpace(space);
 
-        PreparedStatement stmt;
+        GridCacheTwoStepQuery twoStepQry;
+        List<GridQueryFieldMetadata> meta;
 
-        try {
-            stmt = c.prepareStatement(sqlQry);
-        }
-        catch (SQLException e) {
-            throw new CacheException("Failed to parse query: " + sqlQry, e);
-        }
+        final T3<String, String, Boolean> cachedQryKey = new T3<>(space, sqlQry, qry.isCollocated());
+        TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
-        try {
-            bindParameters(stmt, F.asList(qry.getArgs()));
-        }
-        catch (IgniteCheckedException e) {
-            throw new CacheException("Failed to bind parameters: [qry=" + sqlQry + ", params=" +
-                Arrays.deepToString(qry.getArgs()) + "]", e);
+        if (cachedQry != null) {
+            twoStepQry = cachedQry.twoStepQry.copy(qry.getArgs());
+            meta = cachedQry.meta;
         }
+        else {
+            PreparedStatement stmt;
 
-        GridCacheTwoStepQuery twoStepQry;
-        List<GridQueryFieldMetadata> meta;
+            try {
+                // Do not cache this statement because the whole two step query object will be cached later on.
+                stmt = prepareStatement(c, sqlQry, false);
+            }
+            catch (SQLException e) {
+                throw new CacheException("Failed to parse query: " + sqlQry, e);
+            }
+            try {
+                try {
+                    bindParameters(stmt, F.asList(qry.getArgs()));
+                }
+                catch (IgniteCheckedException e) {
+                    throw new CacheException("Failed to bind parameters: [qry=" + sqlQry + ", params=" +
+                        Arrays.deepToString(qry.getArgs()) + "]", e);
+                }
 
-        try {
-            twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), qry.isCollocated());
+                try {
+                    twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), qry.isCollocated());
 
-            meta = meta(stmt.getMetaData());
-        }
-        catch (SQLException e) {
-            throw new CacheException(e);
-        }
-        finally {
-            U.close(stmt, log);
+                    meta = meta(stmt.getMetaData());
+                }
+                catch (SQLException e) {
+                    throw new CacheException(e);
+                }
+            }
+            finally {
+                U.close(stmt, log);
+            }
         }
 
         if (log.isDebugEnabled())
@@ -970,6 +1041,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         cursor.fieldsMeta(meta);
 
+        if (cachedQry == null && !twoStepQry.explain()) {
+            cachedQry = new TwoStepCachedQuery(meta, twoStepQry.copy(null));
+            twoStepCache.putIfAbsent(cachedQryKey, cachedQry);
+        }
+
         return cursor;
     }
 
@@ -1542,6 +1618,31 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * Cached two-step query.
+     */
+    private static final class TwoStepCachedQuery {
+        /** */
+        final List<GridQueryFieldMetadata> meta;
+
+        /** */
+        final GridCacheTwoStepQuery twoStepQry;
+
+        /**
+         * @param meta Fields metadata.
+         * @param twoStepQry Query.
+         */
+        public TwoStepCachedQuery(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
+            this.meta = meta;
+            this.twoStepQry = twoStepQry;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TwoStepCachedQuery.class, this);
+        }
+    }
+
+    /**
      * Wrapper to store connection and flag is schema set or not.
      */
     private static class ConnectionWrapper {
@@ -1889,7 +1990,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @throws IgniteCheckedException If failed.
          */
         protected FieldsIterator(ResultSet data) throws IgniteCheckedException {
-            super(data);
+            super(data, false);
         }
 
         /** {@inheritDoc} */
@@ -1914,7 +2015,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @throws IgniteCheckedException If failed.
          */
         protected KeyValIterator(ResultSet data) throws IgniteCheckedException {
-            super(data);
+            super(data, false);
         }
 
         /** {@inheritDoc} */
@@ -2091,6 +2192,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         /** */
         private final boolean preferSwapVal;
 
+        /** */
+        private final boolean snapshotableIdx;
+
+        /** */
+        private final GridQueryProperty[] props;
+
         /**
          * @param type Type descriptor.
          * @param schema Schema.
@@ -2120,7 +2227,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             keyType = DataType.getTypeFromClass(type.keyClass());
             valType = DataType.getTypeFromClass(type.valueClass());
 
+            props = new GridQueryProperty[fields.length];
+
+            for (int i = 0; i < fields.length; i++) {
+                GridQueryProperty p = type.property(fields[i]);
+
+                assert p != null : fields[i];
+
+                props[i] = p;
+            }
+
             preferSwapVal = schema.ccfg.getMemoryMode() == CacheMemoryMode.OFFHEAP_TIERED;
+            snapshotableIdx = schema.ccfg.isSnapshotableIndex() || schema.offheap != null;
         }
 
         /** {@inheritDoc} */
@@ -2274,7 +2392,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         /** {@inheritDoc} */
         @Override public Object columnValue(Object key, Object val, int col) {
             try {
-                return type.value(fields[col], key, val);
+                return props[col].value(key, val);
             }
             catch (IgniteCheckedException e) {
                 throw DbException.convert(e);
@@ -2298,5 +2416,40 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         @Override public boolean preferSwapValue() {
             return preferSwapVal;
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean snapshotableIndex() {
+            return snapshotableIdx;
+        }
+    }
+
+    /**
+     * Statement cache.
+     */
+    private static class StatementCache extends LinkedHashMap<String, PreparedStatement> {
+        /** */
+        private int size;
+
+        /**
+         * @param size Size.
+         */
+        private StatementCache(int size) {
+            super(size, (float)0.75, true);
+
+            this.size = size;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected boolean removeEldestEntry(Map.Entry<String, PreparedStatement> eldest) {
+            boolean rmv = size() > size;
+
+            if (rmv) {
+                PreparedStatement stmt = eldest.getValue();
+
+                U.closeQuiet(stmt);
+            }
+
+            return rmv;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
index ed3ff7a..80dcfcb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java
@@ -116,4 +116,9 @@ public interface GridH2RowDescriptor extends GridOffHeapSmartPointerFactory<Grid
      * @return {@code True} if should check swap value before offheap.
      */
     public boolean preferSwapValue();
+
+    /**
+     * @return {@code True} if index should support snapshots.
+     */
+    public boolean snapshotableIndex();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index bf318b2..f1e5b16 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -54,6 +54,7 @@ import org.h2.table.TableFilter;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
+import org.jsr166.LongAdder8;
 
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VAL_COL;
 
@@ -79,6 +80,12 @@ public class GridH2Table extends TableBase {
     /** */
     private volatile Object[] actualSnapshot;
 
+    /** */
+    private final LongAdder8 size = new LongAdder8();
+
+    /** */
+    private final boolean snapshotEnabled;
+
     /**
      * Creates table.
      *
@@ -101,10 +108,12 @@ public class GridH2Table extends TableBase {
         assert idxs != null;
         assert idxs.size() >= 1;
 
-        lock =  new ReentrantReadWriteLock();
+        lock = new ReentrantReadWriteLock();
 
         // Add scan index at 0 which is required by H2.
         idxs.add(0, new ScanIndex(index(0)));
+
+        snapshotEnabled = desc == null || desc.snapshotableIndex();
     }
 
     /** {@inheritDoc} */
@@ -164,7 +173,7 @@ public class GridH2Table extends TableBase {
 
         GridUnsafeMemory mem = desc.memory();
 
-        lock.readLock().lock();
+        readLock();
 
         if (mem != null)
             desc.guard().begin();
@@ -183,7 +192,7 @@ public class GridH2Table extends TableBase {
             return true;
         }
         finally {
-            lock.readLock().unlock();
+            readUnlock();
 
             if (mem != null)
                 desc.guard().end();
@@ -209,6 +218,9 @@ public class GridH2Table extends TableBase {
             }
         }
 
+        if (!snapshotEnabled)
+            return;
+
         Object[] snapshot;
 
         for (long waitTime = 100;; waitTime *= 2) { // Increase wait time to avoid starvation.
@@ -297,16 +309,14 @@ public class GridH2Table extends TableBase {
      * Closes table and releases resources.
      */
     public void close() {
-        Lock l = lock.writeLock();
-
-        l.lock();
+        writeLock();
 
         try {
             for (int i = 1, len = idxs.size(); i < len; i++)
                 index(i).close(null);
         }
         finally {
-            l.unlock();
+            writeUnlock();
         }
     }
 
@@ -363,7 +373,7 @@ public class GridH2Table extends TableBase {
         // getting updated from different threads with different rows with the same key is impossible.
         GridUnsafeMemory mem = desc == null ? null : desc.memory();
 
-        lock.readLock().lock();
+        readLock();
 
         if (mem != null)
             desc.guard().begin();
@@ -379,6 +389,8 @@ public class GridH2Table extends TableBase {
 
                     kvOld.onUnswap(kvOld.getValue(VAL_COL), true);
                 }
+                else if (old == null)
+                    size.increment();
 
                 int len = idxs.size();
 
@@ -414,6 +426,8 @@ public class GridH2Table extends TableBase {
                 }
 
                 if (old != null) {
+                    size.decrement();
+
                     // Remove row from all indexes.
                     // Start from 2 because 0 - Scan (don't need to update), 1 - PK (already updated).
                     for (int i = 2, len = idxs.size(); i < len; i++) {
@@ -432,7 +446,7 @@ public class GridH2Table extends TableBase {
             return true;
         }
         finally {
-            lock.readLock().unlock();
+            readUnlock();
 
             if (mem != null)
                 desc.guard().end();
@@ -469,6 +483,9 @@ public class GridH2Table extends TableBase {
      * Rebuilds all indexes of this table.
      */
     public void rebuildIndexes() {
+        if (!snapshotEnabled)
+            return;
+
         GridUnsafeMemory memory = desc == null ? null : desc.memory();
 
         lock.writeLock().lock();
@@ -579,7 +596,7 @@ public class GridH2Table extends TableBase {
 
     /** {@inheritDoc} */
     @Override public long getRowCountApproximation() {
-        return getUniqueIndex().getRowCountApproximation();
+        return size.longValue();
     }
 
     /** {@inheritDoc} */
@@ -605,6 +622,38 @@ public class GridH2Table extends TableBase {
     }
 
     /**
+     *
+     */
+    private void readLock() {
+        if (snapshotEnabled)
+            lock.readLock().lock();
+    }
+
+    /**
+     *
+     */
+    private void readUnlock() {
+        if (snapshotEnabled)
+            lock.readLock().unlock();
+    }
+
+    /**
+     *
+     */
+    private void writeLock() {
+        if (snapshotEnabled)
+            lock.writeLock().lock();
+    }
+
+    /**
+     *
+     */
+    private void writeUnlock() {
+        if (snapshotEnabled)
+            lock.writeLock().unlock();
+    }
+
+    /**
      * H2 Table engine.
      */
     @SuppressWarnings({"PublicInnerClass", "FieldAccessedSynchronizedAndUnsynchronized"})

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 387c58b..28adeee 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -23,6 +23,7 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.NavigableMap;
 import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import org.apache.ignite.internal.util.GridEmptyIterator;
 import org.apache.ignite.internal.util.offheap.unsafe.GridOffHeapSnapTreeMap;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
@@ -51,8 +52,10 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     protected final ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> tree;
 
     /** */
-    private final ThreadLocal<ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row>> snapshot =
-        new ThreadLocal<>();
+    private final ThreadLocal<ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row>> snapshot = new ThreadLocal<>();
+
+    /** */
+    private final boolean snapshotEnabled;
 
     /**
      * Constructor with index initialization.
@@ -81,40 +84,68 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
         final GridH2RowDescriptor desc = tbl.rowDescriptor();
 
-        tree = desc == null || desc.memory() == null ? new SnapTreeMap<GridSearchRowPointer, GridH2Row>(this) {
-            @Override protected void afterNodeUpdate_nl(Node<GridSearchRowPointer, GridH2Row> node, Object val) {
-                if (val != null)
-                    node.key = (GridSearchRowPointer)val;
-            }
+        if (desc == null || desc.memory() == null) {
+            snapshotEnabled = desc == null || desc.snapshotableIndex();
 
-            @Override protected Comparable<? super GridSearchRowPointer> comparable(Object key) {
-                if (key instanceof ComparableRow)
-                    return (Comparable<? super SearchRow>)key;
+            if (snapshotEnabled) {
+                tree = new SnapTreeMap<GridSearchRowPointer, GridH2Row>(this) {
+                    @Override protected void afterNodeUpdate_nl(Node<GridSearchRowPointer, GridH2Row> node, Object val) {
+                        if (val != null)
+                            node.key = (GridSearchRowPointer)val;
+                    }
 
-                return super.comparable(key);
+                    @Override protected Comparable<? super GridSearchRowPointer> comparable(Object key) {
+                        if (key instanceof ComparableRow)
+                            return (Comparable<? super SearchRow>)key;
+
+                        return super.comparable(key);
+                    }
+                };
             }
-        } : new GridOffHeapSnapTreeMap<GridSearchRowPointer, GridH2Row>(desc, desc, desc.memory(), desc.guard(), this) {
-            @Override protected void afterNodeUpdate_nl(long node, GridH2Row val) {
-                final long oldKey = keyPtr(node);
+            else {
+                tree = new ConcurrentSkipListMap<>(
+                        new Comparator<GridSearchRowPointer>() {
+                            @Override public int compare(GridSearchRowPointer o1, GridSearchRowPointer o2) {
+                                if (o1 instanceof ComparableRow)
+                                    return ((ComparableRow)o1).compareTo(o2);
 
-                if (val != null) {
-                    key(node, val);
+                                if (o2 instanceof ComparableRow)
+                                    return -((ComparableRow)o2).compareTo(o1);
 
-                    guard.finalizeLater(new Runnable() {
-                        @Override public void run() {
-                            desc.createPointer(oldKey).decrementRefCount();
+                                return compareRows(o1, o2);
+                            }
                         }
-                    });
-                }
+                );
             }
+        }
+        else {
+            assert desc.snapshotableIndex() : desc;
 
-            @Override protected Comparable<? super GridSearchRowPointer> comparable(Object key) {
-                if (key instanceof ComparableRow)
-                    return (Comparable<? super SearchRow>)key;
+            snapshotEnabled = true;
 
-                return super.comparable(key);
-            }
-        };
+            tree = new GridOffHeapSnapTreeMap<GridSearchRowPointer, GridH2Row>(desc, desc, desc.memory(), desc.guard(), this) {
+                @Override protected void afterNodeUpdate_nl(long node, GridH2Row val) {
+                    final long oldKey = keyPtr(node);
+
+                    if (val != null) {
+                        key(node, val);
+
+                        guard.finalizeLater(new Runnable() {
+                            @Override public void run() {
+                                desc.createPointer(oldKey).decrementRefCount();
+                            }
+                        });
+                    }
+                }
+
+                @Override protected Comparable<? super GridSearchRowPointer> comparable(Object key) {
+                    if (key instanceof ComparableRow)
+                        return (Comparable<? super SearchRow>)key;
+
+                    return super.comparable(key);
+                }
+            };
+        }
     }
 
     /**
@@ -133,6 +164,9 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      */
     @SuppressWarnings("unchecked")
     @Override public Object takeSnapshot(@Nullable Object s) {
+        if (!snapshotEnabled)
+            return null;
+
         assert snapshot.get() == null;
 
         if (s == null)
@@ -148,6 +182,9 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      * Releases snapshot for current thread.
      */
     @Override public void releaseSnapshot() {
+        if (!snapshotEnabled)
+            return;
+
         ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> s = snapshot.get();
 
         snapshot.remove();
@@ -160,6 +197,9 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      * @return Snapshot for current thread if there is one.
      */
     private ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> treeForRead() {
+        if (!snapshotEnabled)
+            return tree;
+
         ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> res = snapshot.get();
 
         if (res == null)
@@ -199,7 +239,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
     /** {@inheritDoc} */
     @Override public long getRowCountApproximation() {
-        return tree.size();
+        return table.getRowCountApproximation();
     }
 
     /** {@inheritDoc} */
@@ -372,7 +412,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      * Comparable row with bias. Will be used for queries to have correct bounds (in case of multicolumn index
      * and query on few first columns we will multiple equal entries in tree).
      */
-    private class ComparableRow implements GridSearchRowPointer, Comparable<SearchRow> {
+    private final class ComparableRow implements GridSearchRowPointer, Comparable<SearchRow> {
         /** */
         private final SearchRow row;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 7b89824..0c9c8fe 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.util.typedef.F;
 import org.h2.jdbc.JdbcPreparedStatement;
+import org.h2.util.IntArray;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG;
@@ -224,13 +225,23 @@ public class GridSqlQuerySplitter {
             rdcQry.distinct(true);
         }
 
-        // Build resulting two step query.
-        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, new GridCacheSqlQuery(rdcQry.getSQL(),
-            findParams(rdcQry, params, new ArrayList<>()).toArray()));
+        IntArray paramIdxs = new IntArray(params.length);
+
+        GridCacheSqlQuery rdc = new GridCacheSqlQuery(rdcQry.getSQL(),
+            findParams(rdcQry, params, new ArrayList<>(), paramIdxs).toArray());
+
+        rdc.parameterIndexes(toIntArray(paramIdxs));
+
+        paramIdxs = new IntArray(params.length);
 
-        res.addMapQuery(new GridCacheSqlQuery(mapQry.getSQL(),
-            findParams(mapQry, params, new ArrayList<>(params.length)).toArray())
-            .columns(collectColumns(mapExps)));
+        GridCacheSqlQuery map = new GridCacheSqlQuery(mapQry.getSQL(),
+            findParams(mapQry, params, new ArrayList<>(params.length), paramIdxs).toArray())
+            .columns(collectColumns(mapExps));
+
+        map.parameterIndexes(toIntArray(paramIdxs));
+
+        // Build resulting two step query.
+        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, rdc, rdcQry.simpleQuery()).addMapQuery(map);
 
         res.explain(explain);
 
@@ -238,6 +249,16 @@ public class GridSqlQuerySplitter {
     }
 
     /**
+     * @param arr Integer array.
+     * @return Primitive int array.
+     */
+    private static int[] toIntArray(IntArray arr) {
+        int[] res = new int[arr.size()];
+        arr.toArray(res);
+        return res;
+    }
+
+    /**
      * @param cols Columns from SELECT clause.
      * @return Map of columns with types.
      */
@@ -341,19 +362,21 @@ public class GridSqlQuerySplitter {
      * @param qry Select.
      * @param params Parameters.
      * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
      * @return Extracted parameters list.
      */
-    private static List<Object> findParams(GridSqlQuery qry, Object[] params, ArrayList<Object> target) {
+    private static List<Object> findParams(GridSqlQuery qry, Object[] params, ArrayList<Object> target,
+        IntArray paramIdxs) {
         if (qry instanceof GridSqlSelect)
-            return findParams((GridSqlSelect)qry, params, target);
+            return findParams((GridSqlSelect)qry, params, target, paramIdxs);
 
         GridSqlUnion union = (GridSqlUnion)qry;
 
-        findParams(union.left(), params, target);
-        findParams(union.right(), params, target);
+        findParams(union.left(), params, target, paramIdxs);
+        findParams(union.right(), params, target, paramIdxs);
 
-        findParams(qry.limit(), params, target);
-        findParams(qry.offset(), params, target);
+        findParams(qry.limit(), params, target, paramIdxs);
+        findParams(qry.offset(), params, target, paramIdxs);
 
         return target;
     }
@@ -362,22 +385,24 @@ public class GridSqlQuerySplitter {
      * @param qry Select.
      * @param params Parameters.
      * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
      * @return Extracted parameters list.
      */
-    private static List<Object> findParams(GridSqlSelect qry, Object[] params, ArrayList<Object> target) {
+    private static List<Object> findParams(GridSqlSelect qry, Object[] params, ArrayList<Object> target,
+        IntArray paramIdxs) {
         if (params.length == 0)
             return target;
 
         for (GridSqlElement el : qry.columns(false))
-            findParams(el, params, target);
+            findParams(el, params, target, paramIdxs);
 
-        findParams(qry.from(), params, target);
-        findParams(qry.where(), params, target);
+        findParams(qry.from(), params, target, paramIdxs);
+        findParams(qry.where(), params, target, paramIdxs);
 
         // Don't search in GROUP BY and HAVING since they expected to be in select list.
 
-        findParams(qry.limit(), params, target);
-        findParams(qry.offset(), params, target);
+        findParams(qry.limit(), params, target, paramIdxs);
+        findParams(qry.offset(), params, target, paramIdxs);
 
         return target;
     }
@@ -386,15 +411,17 @@ public class GridSqlQuerySplitter {
      * @param el Element.
      * @param params Parameters.
      * @param target Extracted parameters.
+     * @param paramIdxs Parameter indexes.
      */
-    private static void findParams(@Nullable GridSqlElement el, Object[] params, ArrayList<Object> target) {
+    private static void findParams(@Nullable GridSqlElement el, Object[] params, ArrayList<Object> target,
+        IntArray paramIdxs) {
         if (el == null)
             return;
 
         if (el instanceof GridSqlParameter) {
             // H2 Supports queries like "select ?5" but first 4 non-existing parameters are need to be set to any value.
             // Here we will set them to NULL.
-            int idx = ((GridSqlParameter)el).index();
+            final int idx = ((GridSqlParameter)el).index();
 
             while (target.size() < idx)
                 target.add(null);
@@ -409,12 +436,14 @@ public class GridSqlQuerySplitter {
                 target.add(param);
             else
                 target.set(idx, param);
+
+            paramIdxs.add(idx);
         }
         else if (el instanceof GridSqlSubquery)
-            findParams(((GridSqlSubquery)el).select(), params, target);
+            findParams(((GridSqlSubquery)el).select(), params, target, paramIdxs);
         else
             for (GridSqlElement child : el)
-                findParams(child, params, target);
+                findParams(child, params, target, paramIdxs);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
index e537ace..e190c87 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlSelect.java
@@ -103,6 +103,33 @@ public class GridSqlSelect extends GridSqlQuery {
     }
 
     /**
+     * @return {@code True} if this simple SQL query like 'SELECT A, B, C from SOME_TABLE' without any conditions
+     *      and expressions.
+     */
+    public boolean simpleQuery() {
+        boolean simple = !distinct &&
+            from instanceof GridSqlTable &&
+            where == null &&
+            grpCols == null &&
+            havingCol < 0 &&
+            sort.isEmpty() &&
+            limit == null &&
+            offset == null;
+
+        if (simple) {
+            for (GridSqlElement expression : columns(true)) {
+                if (expression instanceof GridSqlAlias)
+                    expression = expression.child();
+
+                if (!(expression instanceof GridSqlColumn))
+                    return false;
+            }
+        }
+
+        return simple;
+    }
+
+    /**
      * @param buff Statement builder.
      * @param expression Alias expression.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index b4e1932..21541ec 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -451,8 +451,11 @@ public class GridMapQueryExecutor {
             int i = 0;
 
             for (GridCacheSqlQuery qry : qrys) {
-                ResultSet rs = h2.executeSqlQueryWithTimer(req.space(), h2.connectionForSpace(req.space()), qry.query(),
-                    F.asList(qry.parameters()));
+                ResultSet rs = h2.executeSqlQueryWithTimer(req.space(),
+                    h2.connectionForSpace(req.space()),
+                    qry.query(),
+                    F.asList(qry.parameters()),
+                    true);
 
                 if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
                     ctx.event().record(new CacheQueryExecutedEvent<>(
@@ -820,17 +823,7 @@ public class GridMapQueryExecutor {
 
             closed = true;
 
-            Statement stmt;
-
-            try {
-                stmt = rs.getStatement();
-            }
-            catch (SQLException e) {
-                throw new IllegalStateException(e); // Must not happen.
-            }
-
             U.close(rs, log);
-            U.close(stmt, log);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
index 7f8caed..12c2240 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -80,6 +80,13 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /**
+     *
+     */
+    protected GridMergeIndex() {
+        // No-op.
+    }
+
+    /**
      * @return Return source nodes for this merge index.
      */
     public Set<UUID> sources() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
index 5f5eb25..501480a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
@@ -47,6 +47,20 @@ public class GridMergeIndexUnsorted extends GridMergeIndex {
         super(tbl, name, IndexType.createScan(false), IndexColumn.wrap(tbl.getColumns()));
     }
 
+    /**
+     * @return Dummy index instance.
+     */
+    public static GridMergeIndexUnsorted createDummy() {
+        return new GridMergeIndexUnsorted();
+    }
+
+    /**
+     *
+     */
+    private GridMergeIndexUnsorted() {
+        // No-op.
+    }
+
     /** {@inheritDoc} */
     @Override protected void addPage0(GridResultPage page) {
         assert page.rowsInPage() > 0 || page.isLast() || page.isFail();

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index f515a78..1d4fa30 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -79,10 +79,12 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.h2.command.ddl.CreateTableData;
 import org.h2.engine.Session;
+import org.h2.index.Cursor;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.jdbc.JdbcResultSet;
 import org.h2.jdbc.JdbcStatement;
 import org.h2.result.ResultInterface;
+import org.h2.result.Row;
 import org.h2.table.Column;
 import org.h2.util.IntArray;
 import org.h2.value.Value;
@@ -184,8 +186,8 @@ public class GridReduceQueryExecutor {
                 UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
                 for (QueryRun r : runs.values()) {
-                    for (GridMergeTable tbl : r.tbls) {
-                        if (tbl.getScanIndex(null).hasSource(nodeId)) {
+                    for (GridMergeIndex idx : r.idxs) {
+                        if (idx.hasSource(nodeId)) {
                             handleNodeLeft(r, nodeId);
 
                             break;
@@ -270,7 +272,7 @@ public class GridReduceQueryExecutor {
 
         final int pageSize = r.pageSize;
 
-        GridMergeIndex idx = r.tbls.get(msg.query()).getScanIndex(null);
+        GridMergeIndex idx = r.idxs.get(msg.query());
 
         GridResultPage page;
 
@@ -468,7 +470,7 @@ public class GridReduceQueryExecutor {
 
             r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();
 
-            r.tbls = new ArrayList<>(qry.mapQueries().size());
+            r.idxs = new ArrayList<>(qry.mapQueries().size());
 
             String space = cctx.name();
 
@@ -501,7 +503,7 @@ public class GridReduceQueryExecutor {
             assert !nodes.isEmpty();
 
             if (cctx.isReplicated() || qry.explain()) {
-                assert qry.explain() || !nodes.contains(ctx.cluster().get().localNode()) :
+                assert qry.explain() || !nodes.contains(ctx.discovery().localNode()) :
                     "We must be on a client node.";
 
                 // Select random data node to run query on a replicated data or get EXPLAIN PLAN from a single node.
@@ -510,27 +512,35 @@ public class GridReduceQueryExecutor {
 
             int tblIdx = 0;
 
+            final boolean skipMergeTbl = !qry.explain() && qry.skipMergeTable();
+
             for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
-                GridMergeTable tbl;
+                GridMergeIndex idx;
 
-                try {
-                    tbl = createMergeTable(r.conn, mapQry, qry.explain());
-                }
-                catch (IgniteCheckedException e) {
-                    throw new IgniteException(e);
-                }
+                if (!skipMergeTbl) {
+                    GridMergeTable tbl;
+
+                    try {
+                        tbl = createMergeTable(r.conn, mapQry, qry.explain());
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IgniteException(e);
+                    }
 
-                GridMergeIndex idx = tbl.getScanIndex(null);
+                    idx = tbl.getScanIndex(null);
+
+                    fakeTable(r.conn, tblIdx++).setInnerTable(tbl);
+                }
+                else
+                    idx = GridMergeIndexUnsorted.createDummy();
 
                 for (ClusterNode node : nodes)
                     idx.addSource(node.id());
 
-                r.tbls.add(tbl);
-
-                fakeTable(r.conn, tblIdx++).setInnerTable(tbl);
+                r.idxs.add(idx);
             }
 
-            r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
+            r.latch = new CountDownLatch(r.idxs.size() * nodes.size());
 
             runs.put(qryReqId, r);
 
@@ -586,19 +596,52 @@ public class GridReduceQueryExecutor {
                 else // Send failed.
                     retry = true;
 
-                ResultSet res = null;
+                Iterator<List<?>> resIter = null;
 
                 if (!retry) {
                     if (qry.explain())
                         return explainPlan(r.conn, space, qry);
 
-                    GridCacheSqlQuery rdc = qry.reduceQuery();
+                    if (skipMergeTbl) {
+                        List<List<?>> res = new ArrayList<>();
+
+                        assert r.idxs.size() == 1 : r.idxs;
 
-                    res = h2.executeSqlQueryWithTimer(space, r.conn, rdc.query(), F.asList(rdc.parameters()));
+                        GridMergeIndex idx = r.idxs.get(0);
+
+                        Cursor cur = idx.findInStream(null, null);
+
+                        while (cur.next()) {
+                            Row row = cur.get();
+
+                            int cols = row.getColumnCount();
+
+                            List<Object> resRow  = new ArrayList<>(cols);
+
+                            for (int c = 0; c < cols; c++)
+                                resRow.add(row.getValue(c).getObject());
+
+                            res.add(resRow);
+                        }
+
+                        resIter = res.iterator();
+                    }
+                    else {
+                        GridCacheSqlQuery rdc = qry.reduceQuery();
+
+                        // Statement caching is prohibited here because we can't guarantee correct merge index reuse.
+                        ResultSet res = h2.executeSqlQueryWithTimer(space,
+                            r.conn,
+                            rdc.query(),
+                            F.asList(rdc.parameters()),
+                            false);
+
+                        resIter = new Iter(res);
+                    }
                 }
 
-                for (GridMergeTable tbl : r.tbls) {
-                    if (!tbl.getScanIndex(null).fetchedAll()) // We have to explicitly cancel queries on remote nodes.
+                for (GridMergeIndex idx : r.idxs) {
+                    if (!idx.fetchedAll()) // We have to explicitly cancel queries on remote nodes.
                         send(nodes, new GridQueryCancelRequest(qryReqId), null);
                 }
 
@@ -609,7 +652,7 @@ public class GridReduceQueryExecutor {
                     continue;
                 }
 
-                return new GridQueryCacheObjectsIterator(new Iter(res), cctx, keepPortable);
+                return new GridQueryCacheObjectsIterator(resIter, cctx, keepPortable);
             }
             catch (IgniteCheckedException | RuntimeException e) {
                 U.closeQuiet(r.conn);
@@ -633,8 +676,10 @@ public class GridReduceQueryExecutor {
                 if (!runs.remove(qryReqId, r))
                     U.warn(log, "Query run was already removed: " + qryReqId);
 
-                for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++)
-                    fakeTable(null, i).setInnerTable(null); // Drop all merge tables.
+                if (!skipMergeTbl) {
+                    for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++)
+                        fakeTable(null, i).setInnerTable(null); // Drop all merge tables.
+                }
             }
         }
     }
@@ -941,7 +986,7 @@ public class GridReduceQueryExecutor {
         List<List<?>> lists = new ArrayList<>();
 
         for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) {
-            ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + table(i), null);
+            ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "SELECT PLAN FROM " + table(i), null, false);
 
             lists.add(F.asList(getPlan(rs)));
         }
@@ -956,7 +1001,11 @@ public class GridReduceQueryExecutor {
 
         GridCacheSqlQuery rdc = qry.reduceQuery();
 
-        ResultSet rs = h2.executeSqlQueryWithTimer(space, c, "EXPLAIN " + rdc.query(), F.asList(rdc.parameters()));
+        ResultSet rs = h2.executeSqlQueryWithTimer(space,
+            c,
+            "EXPLAIN " + rdc.query(),
+            F.asList(rdc.parameters()),
+            false);
 
         lists.add(F.asList(getPlan(rs)));
 
@@ -1013,7 +1062,7 @@ public class GridReduceQueryExecutor {
         }
 
         if (locNodeFound) // Local node goes the last to allow parallel execution.
-            h2.mapQueryExecutor().onMessage(ctx.localNodeId(), copy(msg, ctx.cluster().get().localNode(), partsMap));
+            h2.mapQueryExecutor().onMessage(ctx.localNodeId(), copy(msg, ctx.discovery().localNode(), partsMap));
 
         return ok;
     }
@@ -1120,7 +1169,7 @@ public class GridReduceQueryExecutor {
      */
     private static class QueryRun {
         /** */
-        private List<GridMergeTable> tbls;
+        private List<GridMergeIndex> idxs;
 
         /** */
         private CountDownLatch latch;
@@ -1148,8 +1197,8 @@ public class GridReduceQueryExecutor {
             while (latch.getCount() != 0) // We don't need to wait for all nodes to reply.
                 latch.countDown();
 
-            for (GridMergeTable tbl : tbls) // Fail all merge indexes.
-                tbl.getScanIndex(null).fail(nodeId);
+            for (GridMergeIndex idx : idxs) // Fail all merge indexes.
+                idx.fail(nodeId);
         }
 
         /**
@@ -1162,8 +1211,8 @@ public class GridReduceQueryExecutor {
             while (latch.getCount() != 0) // We don't need to wait for all nodes to reply.
                 latch.countDown();
 
-            for (GridMergeTable tbl : tbls) // Fail all merge indexes.
-                tbl.getScanIndex(null).fail(e);
+            for (GridMergeIndex idx : idxs) // Fail all merge indexes.
+                idx.fail(e);
         }
     }
 
@@ -1179,7 +1228,7 @@ public class GridReduceQueryExecutor {
          * @throws IgniteCheckedException If failed.
          */
         protected Iter(ResultSet data) throws IgniteCheckedException {
-            super(data);
+            super(data, true);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index f3fbf15..c510600 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -156,36 +156,38 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
                 if (i > 0)
                     cc.setName("c" + i);
 
-            cc.setCacheMode(cacheMode());
-            cc.setAtomicityMode(atomicityMode());
-            cc.setNearConfiguration(nearCacheConfiguration());
-            cc.setWriteSynchronizationMode(FULL_SYNC);
-            cc.setCacheStoreFactory(new StoreFactory());
-            cc.setReadThrough(true);
-            cc.setWriteThrough(true);
-            cc.setLoadPreviousValue(true);
-            cc.setRebalanceMode(SYNC);
-            cc.setSwapEnabled(true);
-            cc.setSqlFunctionClasses(SqlFunctions.class);
-            cc.setIndexedTypes(
-                BadHashKeyObject.class, Byte.class,
-                ObjectValue.class, Long.class,
-                Integer.class, Integer.class,
-                Integer.class, String.class,
-                Integer.class, ObjectValue.class,
-                String.class, ObjectValueOther.class,
-                Integer.class, ArrayObject.class,
-                Key.class, GridCacheQueryTestValue.class,
-                UUID.class, Person.class,
-                IgniteCacheReplicatedQuerySelfTest.CacheKey.class, IgniteCacheReplicatedQuerySelfTest.CacheValue.class
-            );
-
-            if (cacheMode() != CacheMode.LOCAL)
-                cc.setAffinity(new RendezvousAffinityFunction());
-
-            // Explicitly set number of backups equal to number of grids.
-            if (cacheMode() == CacheMode.PARTITIONED)
-                cc.setBackups(gridCount());
+                cc.setCacheMode(cacheMode());
+                cc.setAtomicityMode(atomicityMode());
+                cc.setNearConfiguration(nearCacheConfiguration());
+                cc.setWriteSynchronizationMode(FULL_SYNC);
+                cc.setCacheStoreFactory(new StoreFactory());
+                cc.setReadThrough(true);
+                cc.setWriteThrough(true);
+                cc.setLoadPreviousValue(true);
+                cc.setRebalanceMode(SYNC);
+                cc.setSwapEnabled(true);
+                cc.setSqlFunctionClasses(SqlFunctions.class);
+                cc.setIndexedTypes(
+                    BadHashKeyObject.class, Byte.class,
+                    ObjectValue.class, Long.class,
+                    Integer.class, Integer.class,
+                    Integer.class, String.class,
+                    Integer.class, ObjectValue.class,
+                    String.class, ObjectValueOther.class,
+                    Integer.class, ArrayObject.class,
+                    Key.class, GridCacheQueryTestValue.class,
+                    UUID.class, Person.class,
+                    IgniteCacheReplicatedQuerySelfTest.CacheKey.class, IgniteCacheReplicatedQuerySelfTest.CacheValue.class
+                );
+
+                if (cacheMode() != CacheMode.LOCAL)
+                    cc.setAffinity(new RendezvousAffinityFunction());
+
+                // Explicitly set number of backups equal to number of grids.
+                if (cacheMode() == CacheMode.PARTITIONED)
+                    cc.setBackups(gridCount());
+
+                cc.setSnapshotableIndex(snapshotableIndex());
 
                 ccs[i] = cc;
             }
@@ -198,6 +200,13 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
         return c;
     }
 
+    /**
+     * @return {@code True} if index snapshot is enabled.
+     */
+    protected boolean snapshotableIndex() {
+        return false;
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         ignite = startGridsMultiThreaded(gridCount());

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedSnapshotEnabledQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedSnapshotEnabledQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedSnapshotEnabledQuerySelfTest.java
new file mode 100644
index 0000000..9a68b16
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedSnapshotEnabledQuerySelfTest.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.distributed.near;
+
+/**
+ *
+ */
+public class IgniteCachePartitionedSnapshotEnabledQuerySelfTest extends IgniteCachePartitionedQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean snapshotableIndex() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 194fb82..c027b26 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryIndexType;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -505,6 +506,24 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public GridQueryProperty property(final String name) {
+            return new GridQueryProperty() {
+                @Override public Object value(Object key, Object val) throws IgniteCheckedException {
+                    return TypeDesc.this.value(name, key, val);
+                }
+
+                @Override public String name() {
+                    return name;
+                }
+
+                @Override
+                public Class<?> type() {
+                    return Object.class;
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public <T> T value(String field, Object key, Object val) throws IgniteSpiException {
             assert !F.isEmpty(field);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4af461a7/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 6cc2599..0745cd7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheP
 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;
@@ -118,6 +119,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         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);


[14/43] ignite git commit: ignite-1.5 - Fixing tests.

Posted by yz...@apache.org.
ignite-1.5 - Fixing tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 385668b3bd9a77f87c11e39a7d915a54a48e39be
Parents: 7680392
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 19:34:35 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 19:34:35 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/NodeStoppingException.java  | 35 ++++++++++++++++++++
 .../processors/cache/GridCacheIoManager.java    |  4 +++
 .../processors/cache/GridCacheMvccManager.java  |  3 +-
 .../distributed/dht/GridDhtLockFuture.java      | 19 ++++++-----
 .../CacheObjectBinaryProcessorImpl.java         |  6 ++--
 5 files changed, 54 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/385668b3/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java b/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java
new file mode 100644
index 0000000..164983a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ *
+ */
+public class NodeStoppingException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param msg Exception message.
+     */
+    public NodeStoppingException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/385668b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 7b1d749..f7fe5bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
@@ -1036,6 +1037,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         catch (IgniteCheckedException e) {
             cacheMsg.onClassError(e);
         }
+        catch (BinaryObjectException e) {
+            cacheMsg.onClassError(new IgniteCheckedException(e));
+        }
         catch (Error e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, NoClassDefFoundError.class,
                 UnsupportedClassVersionError.class))

http://git-wip-us.apache.org/repos/asf/ignite/blob/385668b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 2449df1..dbc6992 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -34,6 +34,7 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -369,7 +370,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @return Node stop exception.
      */
     private IgniteCheckedException stopError() {
-        return new IgniteCheckedException("Operation has been cancelled (node is stopping).");
+        return new NodeStoppingException("Operation has been cancelled (node is stopping).");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/385668b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 7792de3..491ccd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
@@ -481,7 +482,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     private void onFailed(boolean dist) {
         undoLocks(dist);
 
-        onComplete(false);
+        onComplete(false, false);
     }
 
     /**
@@ -627,7 +628,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             err = t;
         }
 
-        onComplete(false);
+        onComplete(false, false);
     }
 
     /**
@@ -690,7 +691,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     /** {@inheritDoc} */
     @Override public boolean cancel() {
         if (onCancelled())
-            onComplete(false);
+            onComplete(false, false);
 
         return isCancelled();
     }
@@ -720,7 +721,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                 this.err = err;
         }
 
-        return onComplete(success);
+        return onComplete(success, err instanceof NodeStoppingException);
     }
 
     /**
@@ -729,17 +730,17 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      * @param success {@code True} if lock was acquired.
      * @return {@code True} if complete by this operation.
      */
-    private boolean onComplete(boolean success) {
+    private boolean onComplete(boolean success, boolean stopping) {
         if (log.isDebugEnabled())
             log.debug("Received onComplete(..) callback [success=" + success + ", fut=" + this + ']');
 
-        if (!success)
+        if (!success && !stopping)
             undoLocks(true);
 
         if (tx != null)
             cctx.tm().txContext(tx);
 
-        if (err == null)
+        if (err == null && !stopping)
             loadMissingFromStore();
 
         if (super.onDone(success, err)) {
@@ -771,7 +772,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      */
     public void map() {
         if (F.isEmpty(entries)) {
-            onComplete(true);
+            onComplete(true, false);
 
             return;
         }
@@ -1062,7 +1063,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
 
             timedOut = true;
 
-            onComplete(false);
+            onComplete(false, false);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/385668b3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index dd9d43f..ba7d6f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -429,9 +429,9 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         Object obj0 = portableMarsh.unmarshal(arr, null);
 
-        assert obj0 instanceof BinaryObject;
-
-        ((BinaryObjectImpl)obj0).detachAllowed(true);
+        // Possible if a class has writeObject method.
+        if (obj0 instanceof BinaryObject)
+            ((BinaryObjectImpl)obj0).detachAllowed(true);
 
         return obj0;
     }


[39/43] ignite git commit: Removed debug.

Posted by yz...@apache.org.
Removed debug.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 49763619f9220a794d3453944a7fc8dda9ce4d44
Parents: bfb839c
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 13:24:41 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 13:24:41 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheIoManager.java  | 2 --
 .../ignite/internal/processors/cache/GridCacheProcessor.java  | 7 -------
 .../cache/distributed/dht/GridPartitionedSingleGetFuture.java | 1 -
 3 files changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/49763619/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index f7fe5bd..0f006a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -146,8 +146,6 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
 
             if (fut != null && !fut.isDone()) {
-                U.debug(log, "<> Will wait for affinity ready future [fut=" + fut + ", msg=" + msg + ']');
-
                 fut.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> t) {
                         cctx.kernalContext().closure().runLocalSafe(new Runnable() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/49763619/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 6822ded..6654a15 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1049,9 +1049,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         cacheCtx.onStarted();
 
-        U.debug(log, "Started cache [name=" + U.maskName(cfg.getName()) + ", deploymentId=" +
-            cacheCtx.dynamicDeploymentId() + ']');
-
         if (log.isInfoEnabled())
             log.info("Started cache [name=" + U.maskName(cfg.getName()) + ", mode=" + cfg.getCacheMode() + ']');
     }
@@ -1604,10 +1601,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
             return;
 
-        U.debug(log, "prepare cache start [locNodeId=" + ctx.localNodeId() +
-            ", initiatingNodeId=" + initiatingNodeId + ", deploymentId=" + deploymentId + ", topVer=" + topVer +
-            ", name=" + cfg.getName() + ']');
-
         if (affNodeStart || clientNodeStart) {
             if (clientNodeStart && !affNodeStart) {
                 if (nearCfg != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/49763619/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index a5e5d53..2d13c2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -401,7 +401,6 @@ public class GridPartitionedSingleGetFuture extends GridFutureAdapter<Object> im
                     return null;
                 }
                 catch (GridCacheEntryRemovedException ignored) {
-                    U.debug(log, ">>>>>>>>> " + entry);
                     // No-op, will retry.
                 }
             }


[29/43] ignite git commit: IGNITE-2006: Moved "typeId()" method from BinaryObject to BinaryType.

Posted by yz...@apache.org.
IGNITE-2006: Moved "typeId()" method from BinaryObject to BinaryType.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 58b2944b975adb398ce5f468227767dbaaf29dd7
Parents: 041cd38
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 26 12:00:28 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 26 12:00:28 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/binary/BinaryObject.java  |   7 -
 .../org/apache/ignite/binary/BinaryType.java    |   7 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   3 +-
 .../internal/portable/BinaryFieldImpl.java      |   6 +-
 .../internal/portable/BinaryObjectEx.java       | 225 +----------------
 .../internal/portable/BinaryObjectExImpl.java   | 245 +++++++++++++++++++
 .../internal/portable/BinaryObjectImpl.java     |   2 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryTypeImpl.java       |   5 +
 .../ignite/internal/portable/PortableUtils.java |   2 +-
 .../builder/PortableBuilderSerializer.java      |   4 +-
 .../CacheObjectBinaryProcessorImpl.java         |   3 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |  12 +-
 .../portable/BinaryFieldsHeapSelfTest.java      |   2 +-
 .../portable/BinaryFieldsOffheapSelfTest.java   |   2 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   4 +-
 .../BinaryFooterOffsetsHeapSelfTest.java        |   2 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   2 +-
 .../portable/BinaryMarshallerSelfTest.java      |  16 +-
 .../portable/BinaryObjectBuilderSelfTest.java   |  71 +++---
 .../PlatformComputeBinarizableArgTask.java      |   6 +-
 21 files changed, 335 insertions(+), 293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
index 2691c7b..432ace2 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
@@ -100,13 +100,6 @@ import java.util.TreeMap;
  */
 public interface BinaryObject extends Serializable, Cloneable {
     /**
-     * Gets binary object type ID.
-     *
-     * @return Type ID.
-     */
-    public int typeId();
-
-    /**
      * Gets type information for this binary object.
      *
      * @return Binary object type information.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
index 52bb212..710bf55 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
@@ -34,6 +34,13 @@ public interface BinaryType {
     public String typeName();
 
     /**
+     * Gets binary type ID.
+     *
+     * @return Binary type ID.
+     */
+    public int typeId();
+
+    /**
      * Gets collection of all field names for this binary type.
      *
      * @return Collection of all field names for this binary type.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index aa013b9..b837e5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -32,6 +32,7 @@ import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
@@ -270,7 +271,7 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
      */
     @Override protected Object typeIdForObject(Object obj) throws CacheException {
         if (obj instanceof BinaryObject)
-            return ((BinaryObject)obj).typeId();
+            return ((BinaryObjectEx)obj).typeId();
 
         return obj.getClass();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
index 810c820..b471fbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryFieldImpl.java
@@ -66,7 +66,7 @@ public class BinaryFieldImpl implements BinaryField {
 
     /** {@inheritDoc} */
     @Override public boolean exists(BinaryObject obj) {
-        BinaryObjectEx obj0 = (BinaryObjectEx)obj;
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
 
         return fieldOrder(obj0) != PortableSchema.ORDER_NOT_FOUND;
     }
@@ -74,7 +74,7 @@ public class BinaryFieldImpl implements BinaryField {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T> T value(BinaryObject obj) {
-        BinaryObjectEx obj0 = (BinaryObjectEx)obj;
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
 
         int order = fieldOrder(obj0);
 
@@ -87,7 +87,7 @@ public class BinaryFieldImpl implements BinaryField {
      * @param obj Object.
      * @return Field offset.
      */
-    private int fieldOrder(BinaryObjectEx obj) {
+    private int fieldOrder(BinaryObjectExImpl obj) {
         if (typeId != obj.typeId()) {
             throw new BinaryObjectException("Failed to get field because type ID of passed object differs" +
                 " from type ID this " + BinaryField.class.getSimpleName() + " belongs to [expected=" + typeId +

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
index 597fad5..acc8e4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
@@ -17,229 +17,14 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.IdentityHashMap;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryObject;
-import org.jetbrains.annotations.Nullable;
 
 /**
- * Internal portable object interface.
+ * Extended binary object interface.
  */
-public abstract class BinaryObjectEx implements BinaryObject {
+public interface BinaryObjectEx extends BinaryObject {
     /**
-     * @return Length.
+     * @return Type ID.
      */
-    public abstract int length();
-
-    /**
-     * @return Object start.
-     */
-    public abstract int start();
-
-    /**
-     * @return {@code True} if object is array based.
-     */
-    protected abstract boolean hasArray();
-
-    /**
-     * @return Object array if object is array based, otherwise {@code null}.
-     */
-    public abstract byte[] array();
-
-    /**
-     * @return Object offheap address is object is offheap based, otherwise 0.
-     */
-    public abstract long offheapAddress();
-
-    /**
-     * Gets field value.
-     *
-     * @param fieldId Field ID.
-     * @return Field value.
-     * @throws org.apache.ignite.binary.BinaryObjectException In case of any other error.
-     */
-    @Nullable public abstract <F> F field(int fieldId) throws BinaryObjectException;
-
-    /**
-     * Get field by offset.
-     *
-     * @param fieldOffset Field offset.
-     * @return Field value.
-     */
-    @Nullable protected abstract <F> F fieldByOrder(int fieldOffset);
-
-    /**
-     * @param ctx Reader context.
-     * @param fieldName Field name.
-     * @return Field value.
-     */
-    @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
-
-    /**
-     * Get schema ID.
-     *
-     * @return Schema ID.
-     */
-    protected abstract int schemaId();
-
-    /**
-     * Create schema for object.
-     *
-     * @return Schema.
-     */
-    protected abstract PortableSchema createSchema();
-
-    /** {@inheritDoc} */
-    @Override public BinaryObject clone() throws CloneNotSupportedException {
-        return (BinaryObject)super.clone();
-    }
-
-    /** {@inheritDoc} */
-    public boolean equals(Object other) {
-        if (other == this)
-            return true;
-
-        if (other == null)
-            return false;
-
-        if (!(other instanceof BinaryObjectEx))
-            return false;
-
-        BinaryObjectEx otherPo = (BinaryObjectEx)other;
-
-        if (length() != otherPo.length() || typeId() != otherPo.typeId())
-            return false;
-
-        if (hasArray()) {
-            if (otherPo.hasArray()) {
-                int len = length();
-                int end = start() + len;
-
-                byte[] arr = array();
-                byte[] otherArr = otherPo.array();
-
-                for (int i = start(), j = otherPo.start(); i < end; i++, j++) {
-                    if (arr[i] != otherArr[j])
-                        return false;
-                }
-
-                return true;
-            }
-            else {
-                assert otherPo.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array());
-            }
-        }
-        else {
-            assert offheapAddress() > 0;
-
-            if (otherPo.hasArray())
-                return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array());
-            else {
-                assert otherPo.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(offheapAddress() + start(),
-                    otherPo.offheapAddress() + otherPo.start(),
-                    length());
-            }
-        }
-    }
-
-    /**
-     * @param ctx Reader context.
-     * @param handles Handles for already traversed objects.
-     * @return String representation.
-     */
-    private String toString(BinaryReaderHandles ctx, IdentityHashMap<BinaryObject, Integer> handles) {
-        int idHash = System.identityHashCode(this);
-
-        BinaryType meta;
-
-        try {
-            meta = type();
-        }
-        catch (BinaryObjectException ignore) {
-            meta = null;
-        }
-
-        if (meta == null)
-            return BinaryObject.class.getSimpleName() +  " [hash=" + idHash + ", typeId=" + typeId() + ']';
-
-        handles.put(this, idHash);
-
-        SB buf = new SB(meta.typeName());
-
-        if (meta.fieldNames() != null) {
-            buf.a(" [hash=").a(idHash);
-
-            for (String name : meta.fieldNames()) {
-                Object val = field(ctx, name);
-
-                buf.a(", ").a(name).a('=');
-
-                if (val instanceof byte[])
-                    buf.a(Arrays.toString((byte[]) val));
-                else if (val instanceof short[])
-                    buf.a(Arrays.toString((short[])val));
-                else if (val instanceof int[])
-                    buf.a(Arrays.toString((int[])val));
-                else if (val instanceof long[])
-                    buf.a(Arrays.toString((long[])val));
-                else if (val instanceof float[])
-                    buf.a(Arrays.toString((float[])val));
-                else if (val instanceof double[])
-                    buf.a(Arrays.toString((double[])val));
-                else if (val instanceof char[])
-                    buf.a(Arrays.toString((char[])val));
-                else if (val instanceof boolean[])
-                    buf.a(Arrays.toString((boolean[]) val));
-                else if (val instanceof BigDecimal[])
-                    buf.a(Arrays.toString((BigDecimal[])val));
-                else {
-                    if (val instanceof BinaryObjectEx) {
-                        BinaryObjectEx po = (BinaryObjectEx)val;
-
-                        Integer idHash0 = handles.get(val);
-
-                        if (idHash0 != null) {  // Circular reference.
-                            BinaryType meta0 = po.type();
-
-                            assert meta0 != null;
-
-                            buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]");
-                        }
-                        else
-                            buf.a(po.toString(ctx, handles));
-                    }
-                    else
-                        buf.a(val);
-                }
-            }
-
-            buf.a(']');
-        }
-
-        return buf.toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        try {
-            BinaryReaderHandles ctx = new BinaryReaderHandles();
-
-            ctx.put(start(), this);
-
-            return toString(ctx, new IdentityHashMap<BinaryObject, Integer>());
-        }
-        catch (BinaryObjectException e) {
-            throw new IgniteException("Failed to create string representation of portable object.", e);
-        }
-    }
-}
\ No newline at end of file
+    public int typeId();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java
new file mode 100644
index 0000000..7497bd3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectExImpl.java
@@ -0,0 +1,245 @@
+/*
+ * 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.portable;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.IdentityHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Internal portable object interface.
+ */
+public abstract class BinaryObjectExImpl implements BinaryObjectEx {
+    /**
+     * @return Length.
+     */
+    public abstract int length();
+
+    /**
+     * @return Object start.
+     */
+    public abstract int start();
+
+    /**
+     * @return {@code True} if object is array based.
+     */
+    protected abstract boolean hasArray();
+
+    /**
+     * @return Object array if object is array based, otherwise {@code null}.
+     */
+    public abstract byte[] array();
+
+    /**
+     * @return Object offheap address is object is offheap based, otherwise 0.
+     */
+    public abstract long offheapAddress();
+
+    /**
+     * Gets field value.
+     *
+     * @param fieldId Field ID.
+     * @return Field value.
+     * @throws org.apache.ignite.binary.BinaryObjectException In case of any other error.
+     */
+    @Nullable public abstract <F> F field(int fieldId) throws BinaryObjectException;
+
+    /**
+     * Get field by offset.
+     *
+     * @param fieldOffset Field offset.
+     * @return Field value.
+     */
+    @Nullable protected abstract <F> F fieldByOrder(int fieldOffset);
+
+    /**
+     * @param ctx Reader context.
+     * @param fieldName Field name.
+     * @return Field value.
+     */
+    @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
+
+    /**
+     * Get schema ID.
+     *
+     * @return Schema ID.
+     */
+    protected abstract int schemaId();
+
+    /**
+     * Create schema for object.
+     *
+     * @return Schema.
+     */
+    protected abstract PortableSchema createSchema();
+
+    /** {@inheritDoc} */
+    @Override public BinaryObject clone() throws CloneNotSupportedException {
+        return (BinaryObject)super.clone();
+    }
+
+    /** {@inheritDoc} */
+    public boolean equals(Object other) {
+        if (other == this)
+            return true;
+
+        if (other == null)
+            return false;
+
+        if (!(other instanceof BinaryObjectExImpl))
+            return false;
+
+        BinaryObjectExImpl otherPo = (BinaryObjectExImpl)other;
+
+        if (length() != otherPo.length() || typeId() != otherPo.typeId())
+            return false;
+
+        if (hasArray()) {
+            if (otherPo.hasArray()) {
+                int len = length();
+                int end = start() + len;
+
+                byte[] arr = array();
+                byte[] otherArr = otherPo.array();
+
+                for (int i = start(), j = otherPo.start(); i < end; i++, j++) {
+                    if (arr[i] != otherArr[j])
+                        return false;
+                }
+
+                return true;
+            }
+            else {
+                assert otherPo.offheapAddress() > 0;
+
+                return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array());
+            }
+        }
+        else {
+            assert offheapAddress() > 0;
+
+            if (otherPo.hasArray())
+                return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array());
+            else {
+                assert otherPo.offheapAddress() > 0;
+
+                return GridUnsafeMemory.compare(offheapAddress() + start(),
+                    otherPo.offheapAddress() + otherPo.start(),
+                    length());
+            }
+        }
+    }
+
+    /**
+     * @param ctx Reader context.
+     * @param handles Handles for already traversed objects.
+     * @return String representation.
+     */
+    private String toString(BinaryReaderHandles ctx, IdentityHashMap<BinaryObject, Integer> handles) {
+        int idHash = System.identityHashCode(this);
+
+        BinaryType meta;
+
+        try {
+            meta = type();
+        }
+        catch (BinaryObjectException ignore) {
+            meta = null;
+        }
+
+        if (meta == null)
+            return BinaryObject.class.getSimpleName() +  " [hash=" + idHash + ", typeId=" + typeId() + ']';
+
+        handles.put(this, idHash);
+
+        SB buf = new SB(meta.typeName());
+
+        if (meta.fieldNames() != null) {
+            buf.a(" [hash=").a(idHash);
+
+            for (String name : meta.fieldNames()) {
+                Object val = field(ctx, name);
+
+                buf.a(", ").a(name).a('=');
+
+                if (val instanceof byte[])
+                    buf.a(Arrays.toString((byte[]) val));
+                else if (val instanceof short[])
+                    buf.a(Arrays.toString((short[])val));
+                else if (val instanceof int[])
+                    buf.a(Arrays.toString((int[])val));
+                else if (val instanceof long[])
+                    buf.a(Arrays.toString((long[])val));
+                else if (val instanceof float[])
+                    buf.a(Arrays.toString((float[])val));
+                else if (val instanceof double[])
+                    buf.a(Arrays.toString((double[])val));
+                else if (val instanceof char[])
+                    buf.a(Arrays.toString((char[])val));
+                else if (val instanceof boolean[])
+                    buf.a(Arrays.toString((boolean[]) val));
+                else if (val instanceof BigDecimal[])
+                    buf.a(Arrays.toString((BigDecimal[])val));
+                else {
+                    if (val instanceof BinaryObjectExImpl) {
+                        BinaryObjectExImpl po = (BinaryObjectExImpl)val;
+
+                        Integer idHash0 = handles.get(val);
+
+                        if (idHash0 != null) {  // Circular reference.
+                            BinaryType meta0 = po.type();
+
+                            assert meta0 != null;
+
+                            buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]");
+                        }
+                        else
+                            buf.a(po.toString(ctx, handles));
+                    }
+                    else
+                        buf.a(val);
+                }
+            }
+
+            buf.a(']');
+        }
+
+        return buf.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        try {
+            BinaryReaderHandles ctx = new BinaryReaderHandles();
+
+            ctx.put(start(), this);
+
+            return toString(ctx, new IdentityHashMap<BinaryObject, Integer>());
+        }
+        catch (BinaryObjectException e) {
+            throw new IgniteException("Failed to create string representation of portable object.", e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index 7db4b4a..f683ed9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -64,7 +64,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
  * Portable object implementation.
  */
 @IgniteCodeGeneratingFail // Fields arr and start should not be generated by MessageCodeGenerator.
-public final class BinaryObjectImpl extends BinaryObjectEx implements Externalizable, KeyCacheObject {
+public final class BinaryObjectImpl extends BinaryObjectExImpl implements Externalizable, KeyCacheObject {
     /** */
     public static final byte TYPE_BINARY = 100;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index 8b8e0e8..6de0432 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -61,7 +61,7 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
 /**
  *  Portable object implementation over offheap memory
  */
-public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externalizable, CacheObject {
+public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Externalizable, CacheObject {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
index 5f2a91e..c3fee7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryTypeImpl.java
@@ -48,6 +48,11 @@ public class BinaryTypeImpl implements BinaryType {
     }
 
     /** {@inheritDoc} */
+    @Override public int typeId() {
+        return meta.typeId();
+    }
+
+    /** {@inheritDoc} */
     @Override public Collection<String> fieldNames() {
         return meta.fields();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index 5c798b8..0c569cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -1470,7 +1470,7 @@ public class PortableUtils {
 
                 int len = length(in, start);
 
-                BinaryObjectEx po;
+                BinaryObjectExImpl po;
 
                 if (detach) {
                     // In detach mode we simply copy object's content.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
index 0e8eaa4..ee7bd65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.portable.builder;
 
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.BinaryObjectEx;
+import org.apache.ignite.internal.portable.BinaryObjectExImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.*;
@@ -61,7 +61,7 @@ class PortableBuilderSerializer {
             return;
         }
 
-        if (val instanceof BinaryObjectEx) {
+        if (val instanceof BinaryObjectExImpl) {
             if (portableObjToWrapper == null)
                 portableObjToWrapper = new IdentityHashMap<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index 05e9263..d461041 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.portable.BinaryMetadata;
 import org.apache.ignite.internal.portable.BinaryMetadataHandler;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
 import org.apache.ignite.internal.portable.BinaryObjectImpl;
 import org.apache.ignite.internal.portable.BinaryObjectOffheapImpl;
 import org.apache.ignite.internal.portable.BinaryTypeImpl;
@@ -575,7 +576,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         if (obj == null)
             return 0;
 
-        return isPortableObject(obj) ? ((BinaryObject)obj).typeId() : typeId(obj.getClass().getSimpleName());
+        return isPortableObject(obj) ? ((BinaryObjectEx)obj).typeId() : typeId(obj.getClass().getSimpleName());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
index dd08390..ca41358 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
@@ -487,7 +487,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
     private TestContext context(BinaryMarshaller marsh, String fieldName) throws Exception {
         TestObject obj = createObject();
 
-        BinaryObjectEx portObj = toPortable(marsh, obj);
+        BinaryObjectExImpl portObj = toPortable(marsh, obj);
 
         BinaryField field = portObj.type().field(fieldName);
 
@@ -507,8 +507,8 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
         TestObject obj = createObject();
         TestOuterObject outObj = new TestOuterObject(obj);
 
-        BinaryObjectEx portOutObj = toPortable(marsh, outObj);
-        BinaryObjectEx portObj = portOutObj.field("fInner");
+        BinaryObjectExImpl portOutObj = toPortable(marsh, outObj);
+        BinaryObjectExImpl portObj = portOutObj.field("fInner");
 
         assert portObj != null;
 
@@ -534,7 +534,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @return Portable object.
      * @throws Exception If failed.
      */
-    protected abstract BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
+    protected abstract BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
 
     /**
      * Outer test object.
@@ -698,7 +698,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
         public final TestObject obj;
 
         /** Portable object. */
-        public final BinaryObjectEx portObj;
+        public final BinaryObjectExImpl portObj;
 
         /** Field. */
         public final BinaryField field;
@@ -710,7 +710,7 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
          * @param portObj Portable object.
          * @param field Field.
          */
-        public TestContext(TestObject obj, BinaryObjectEx portObj, BinaryField field) {
+        public TestContext(TestObject obj, BinaryObjectExImpl portObj, BinaryField field) {
             this.obj = obj;
             this.portObj = portObj;
             this.field = field;

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
index 53ba212..dbabba5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.marshaller.portable.BinaryMarshaller;
  */
 public class BinaryFieldsHeapSelfTest extends BinaryFieldsAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] bytes = marsh.marshal(obj);
 
         return new BinaryObjectImpl(portableContext(marsh), bytes, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
index 68bfb30..fe9f745 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
@@ -47,7 +47,7 @@ public class BinaryFieldsOffheapSelfTest extends BinaryFieldsAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] arr = marsh.marshal(obj);
 
         long ptr = UNSAFE.allocateMemory(arr.length);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
index 7267b16..d900064 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
@@ -128,7 +128,7 @@ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbst
     private void check(int len) throws Exception {
         TestObject obj = new TestObject(len);
 
-        BinaryObjectEx portObj = toPortable(marsh, obj);
+        BinaryObjectExImpl portObj = toPortable(marsh, obj);
 
         // 1. Test portable object content.
         assert portObj.hasField("field1");
@@ -170,7 +170,7 @@ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbst
      * @return Portable object.
      * @throws Exception If failed.
      */
-    protected abstract BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
+    protected abstract BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception;
 
     /**
      * Test object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
index 471bd44..939f01d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.marshaller.portable.BinaryMarshaller;
  */
 public class BinaryFooterOffsetsHeapSelfTest extends BinaryFooterOffsetsAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] bytes = marsh.marshal(obj);
 
         return new BinaryObjectImpl(ctx, bytes, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
index 7b44b80..f7e743f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
@@ -47,7 +47,7 @@ public class BinaryFooterOffsetsOffheapSelfTest extends BinaryFooterOffsetsAbstr
     }
 
     /** {@inheritDoc} */
-    @Override protected BinaryObjectEx toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
+    @Override protected BinaryObjectExImpl toPortable(BinaryMarshaller marsh, Object obj) throws Exception {
         byte[] arr = marsh.marshal(obj);
 
         long ptr = UNSAFE.allocateMemory(arr.length);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
index 19c40b3..0fbb4c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
@@ -792,9 +792,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
 
-        BinaryObjectEx po1 = marshal(obj1, marsh);
+        BinaryObjectExImpl po1 = marshal(obj1, marsh);
 
-        assertEquals(11111, po1.typeId());
+        assertEquals(11111, po1.type().typeId());
         assertEquals((Integer)10, po1.field(22222));
         assertEquals("str", po1.field(33333));
 
@@ -849,9 +849,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
 
-        BinaryObjectEx po1 = marshal(obj1, marsh);
+        BinaryObjectExImpl po1 = marshal(obj1, marsh);
 
-        assertEquals(11111, po1.typeId());
+        assertEquals(11111, po1.type().typeId());
         assertEquals((Integer)10, po1.field(22222));
         assertEquals("str1", po1.field(33333));
 
@@ -860,9 +860,9 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
 
-        BinaryObjectEx po2 = marshal(obj2, marsh);
+        BinaryObjectExImpl po2 = marshal(obj2, marsh);
 
-        assertEquals(44444, po2.typeId());
+        assertEquals(44444, po2.type().typeId());
         assertEquals((Integer)20, po2.field(55555));
         assertEquals("str2", po2.field(66666));
 
@@ -1774,7 +1774,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
             assertTrue(offheapObj.equals(offheapObj1));
             assertTrue(offheapObj1.equals(offheapObj));
 
-            assertEquals(obj.typeId(), offheapObj.typeId());
+            assertEquals(obj.type().typeId(), offheapObj.type().typeId());
             assertEquals(obj.hashCode(), offheapObj.hashCode());
 
             checkSimpleObjectData(simpleObj, offheapObj);
@@ -1787,7 +1787,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
             obj = (BinaryObjectImpl)offheapObj.heapCopy();
 
-            assertEquals(obj.typeId(), offheapObj.typeId());
+            assertEquals(obj.type().typeId(), offheapObj.type().typeId());
             assertEquals(obj.hashCode(), offheapObj.hashCode());
 
             checkSimpleObjectData(simpleObj, obj);

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
index 7f023f3..4864847 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
@@ -52,6 +52,7 @@ import sun.misc.Unsafe;
 /**
  * Portable builder test.
  */
+@SuppressWarnings("ResultOfMethodCallIgnored")
 public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /** */
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
@@ -135,7 +136,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
@@ -153,7 +154,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals((short)1, po.<Short>field("shortField").shortValue());
@@ -171,7 +172,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1, po.<Integer>field("intField").intValue());
@@ -189,7 +190,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1L, po.<Long>field("longField").longValue());
@@ -207,7 +208,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
@@ -225,7 +226,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
@@ -243,7 +244,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals((char)1, po.<Character>field("charField").charValue());
@@ -261,7 +262,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(po.<Boolean>field("booleanField"));
@@ -279,10 +280,10 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
-        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
+        assertEquals(BigDecimal.TEN, po.<BigDecimal>field("decimalField"));
     }
 
     /**
@@ -297,7 +298,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals("str", po.<String>field("stringField"));
@@ -336,7 +337,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(uuid, po.<UUID>field("uuidField"));
@@ -354,7 +355,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
@@ -372,7 +373,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
@@ -390,7 +391,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
@@ -408,7 +409,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
@@ -426,7 +427,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
@@ -444,7 +445,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
@@ -462,7 +463,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
@@ -480,7 +481,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         boolean[] arr = po.field("booleanArrayField");
@@ -503,7 +504,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
@@ -521,7 +522,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
@@ -568,7 +569,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
@@ -586,7 +587,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1, po.<BinaryObject>field("objectField").<Value>deserialize().i);
@@ -604,7 +605,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         Object[] arr = po.field("objectArrayField");
@@ -627,7 +628,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         List<BinaryObject> list = po.field("collectionField");
@@ -650,7 +651,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         Map<BinaryObject, BinaryObject> map = po.field("mapField");
@@ -677,7 +678,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("class".hashCode(), po.typeId());
+        assertEquals("class".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(111, po.<Integer>field("i").intValue());
@@ -727,7 +728,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
             assertEquals(BinaryObjectOffheapImpl.class, offheapObj.getClass());
 
-            assertEquals("class".hashCode(), offheapObj.typeId());
+            assertEquals("class".hashCode(), offheapObj.type().typeId());
             assertEquals(100, offheapObj.hashCode());
 
             assertEquals(111, offheapObj.<Integer>field("i").intValue());
@@ -762,7 +763,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
         BinaryObject po = builder.build();
 
-        assertEquals("value".hashCode(), po.typeId());
+        assertEquals("value".hashCode(), po.type().typeId());
         assertEquals(100, po.hashCode());
 
         assertEquals(1, po.<Value>deserialize().i);
@@ -856,6 +857,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
+    @SuppressWarnings("unchecked")
     public void testCopyFromInnerObjects() {
         ArrayList<Object> list = new ArrayList<>();
         list.add(new TestObjectAllTypes());
@@ -898,7 +900,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
         assertTrue(builder.getField("plainPortable") instanceof BinaryObject);
 
         TestObjectPlainPortable deserialized = builder.build().deserialize();
-        assertTrue(deserialized.plainPortable instanceof BinaryObject);
+        assertTrue(deserialized.plainPortable != null);
     }
 
     /**
@@ -987,20 +989,21 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
     /**
      * @return Builder.
      */
-    private <T> BinaryObjectBuilder builder(String clsName) {
+    private BinaryObjectBuilder builder(String clsName) {
         return portables().builder(clsName);
     }
 
     /**
      * @return Builder.
      */
-    private <T> BinaryObjectBuilderImpl builder(BinaryObject obj) {
+    private BinaryObjectBuilderImpl builder(BinaryObject obj) {
         return (BinaryObjectBuilderImpl)portables().builder(obj);
     }
 
     /**
      *
      */
+    @SuppressWarnings("UnusedDeclaration")
     private static class CustomIdMapper {
         /** */
         private String str = "a";
@@ -1011,6 +1014,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
     /**
      */
+    @SuppressWarnings("UnusedDeclaration")
     private static class Key {
         /** */
         private int i;
@@ -1049,6 +1053,7 @@ public class BinaryObjectBuilderSelfTest extends GridCommonAbstractTest {
 
     /**
      */
+    @SuppressWarnings("UnusedDeclaration")
     private static class Value {
         /** */
         private int i;

http://git-wip-us.apache.org/repos/asf/ignite/blob/58b2944b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
index 8eba80b..ffae2d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
@@ -19,14 +19,14 @@ package org.apache.ignite.platform;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
+import org.apache.ignite.internal.portable.BinaryObjectEx;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -87,7 +87,7 @@ public class PlatformComputeBinarizableArgTask extends ComputeTaskAdapter<Object
 
         /** {@inheritDoc} */
         @Nullable @Override public Object execute() {
-            BinaryObject arg0 = ((BinaryObject)arg);
+            BinaryObjectEx arg0 = ((BinaryObjectEx)arg);
 
             BinaryType meta = ignite.binary().metadata(arg0.typeId());
 


[20/43] ignite git commit: IGNITE-1990 Added missing classed to classnames-jdk.properties.

Posted by yz...@apache.org.
IGNITE-1990 Added missing classed to classnames-jdk.properties.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 3c42da8b08973fab850a5ebc48b71915ce02f992
Parents: 97270bf
Author: AKuznetsov <ak...@gridgain.com>
Authored: Thu Nov 26 07:21:18 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Thu Nov 26 07:21:18 2015 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheConfiguration.java    |   4 +-
 .../cache/VisorCacheTypeFieldMetadata.java      |  30 ++++-
 .../visor/cache/VisorCacheTypeMetadata.java     | 133 +++++++++++++++++--
 .../META-INF/classnames-jdk.properties          |   2 +
 4 files changed, 152 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3c42da8b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index 073ad22..69eb311 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -138,7 +138,7 @@ public class VisorCacheConfiguration implements Serializable {
         maxConcurrentAsyncOps = ccfg.getMaxConcurrentAsyncOperations();
         memoryMode = ccfg.getMemoryMode();
         interceptor = compactClass(ccfg.getInterceptor());
-        typeMeta = VisorCacheTypeMetadata.list(ccfg.getTypeMetadata());
+        typeMeta = VisorCacheTypeMetadata.list(ccfg.getQueryEntities(), ccfg.getCacheStoreFactory(), ccfg.getTypeMetadata());
         statisticsEnabled = ccfg.isStatisticsEnabled();
         mgmtEnabled = ccfg.isManagementEnabled();
         ldrFactory = compactClass(ccfg.getCacheLoaderFactory());
@@ -350,4 +350,4 @@ public class VisorCacheConfiguration implements Serializable {
     @Override public String toString() {
         return S.toString(VisorCacheConfiguration.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c42da8b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeFieldMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeFieldMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeFieldMetadata.java
index c21354b..323e536 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeFieldMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeFieldMetadata.java
@@ -45,14 +45,30 @@ public class VisorCacheTypeFieldMetadata implements Serializable {
      * @return Data transfer object for given cache field metadata.
      */
     public static VisorCacheTypeFieldMetadata from(CacheTypeFieldMetadata f) {
-        VisorCacheTypeFieldMetadata fieldMetadata = new VisorCacheTypeFieldMetadata();
+        return new VisorCacheTypeFieldMetadata(f.getDatabaseName(), f.getDatabaseType(),
+            f.getJavaName(), U.compact(f.getJavaType().getName()));
+    }
 
-        fieldMetadata.dbName = f.getDatabaseName();
-        fieldMetadata.dbType = f.getDatabaseType();
-        fieldMetadata.javaName = f.getJavaName();
-        fieldMetadata.javaType = U.compact(f.getJavaType().getName());
+    /**
+     * Empty constructor.
+     */
+    public VisorCacheTypeFieldMetadata() {
+        // No-op.
+    }
 
-        return fieldMetadata;
+    /**
+     * Full constructor.
+     *
+     * @param dbName Column name in database.
+     * @param dbType Column JDBC type in database.
+     * @param javaName Field name in java object.
+     * @param javaType Corresponding java type.
+     */
+    public VisorCacheTypeFieldMetadata(String dbName, int dbType, String javaName, String javaType) {
+        this.dbName = dbName;
+        this.dbType = dbType;
+        this.javaName = javaName;
+        this.javaType = javaType;
     }
 
     /**
@@ -82,4 +98,4 @@ public class VisorCacheTypeFieldMetadata implements Serializable {
     public String javaType() {
         return javaType;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c42da8b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
index 2a00ec1..ec7a114 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
@@ -25,10 +25,17 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 import org.apache.ignite.cache.CacheTypeFieldMetadata;
 import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory;
+import org.apache.ignite.cache.store.jdbc.JdbcType;
+import org.apache.ignite.cache.store.jdbc.JdbcTypeField;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 
+import javax.cache.configuration.Factory;
+
 /**
  * Data transfer object for {@link CacheTypeMetadata}.
  */
@@ -77,19 +84,129 @@ public class VisorCacheTypeMetadata implements Serializable {
     private Map<String, LinkedHashMap<String, IgniteBiTuple<String, Boolean>>> grps;
 
     /**
+     * @param qryEntities Collection of query entities.
+     * @param factory Store factory to extract JDBC types info.
      * @param types Cache types metadata configurations.
      * @return Data transfer object for cache type metadata configurations.
      */
-    public static Collection<VisorCacheTypeMetadata> list(Collection<CacheTypeMetadata> types) {
-        if (types == null)
-            return Collections.emptyList();
+    public static Collection<VisorCacheTypeMetadata> list(Collection<QueryEntity> qryEntities, Factory factory,
+        Collection<CacheTypeMetadata> types) {
+        final Collection<VisorCacheTypeMetadata> metas = new ArrayList<>();
+
+        Map<String, VisorCacheTypeMetadata> metaMap =
+                U.newHashMap(qryEntities != null ? qryEntities.size() : 0);
+
+        // Add query entries.
+        if (qryEntities != null)
+            for (QueryEntity qryEntity : qryEntities) {
+                VisorCacheTypeMetadata meta = from(qryEntity);
+
+                metas.add(meta);
+
+                metaMap.put(meta.keyType, meta);
+            }
+
+        // Add JDBC types.
+        if (factory != null && factory instanceof CacheJdbcPojoStoreFactory) {
+             CacheJdbcPojoStoreFactory jdbcFactory = (CacheJdbcPojoStoreFactory) factory;
+
+            for (JdbcType jdbcType : jdbcFactory.getTypes()) {
+                VisorCacheTypeMetadata meta = metaMap.get(jdbcType.getKeyType());
+
+                boolean notFound = meta == null;
+
+                if (notFound) {
+                    meta = new VisorCacheTypeMetadata();
+
+                    meta.keyType = jdbcType.getKeyType();
+                    meta.valType = jdbcType.getValueType();
+
+                    meta.qryFlds = Collections.emptyMap();
+                    meta.ascFlds = Collections.emptyMap();
+                    meta.descFlds = Collections.emptyMap();
+                    meta.txtFlds = Collections.emptyList();
+                    meta.grps = Collections.emptyMap();
+                }
 
-        final Collection<VisorCacheTypeMetadata> cfgs = new ArrayList<>(types.size());
+                meta.dbSchema = jdbcType.getDatabaseSchema();
+                meta.dbTbl = jdbcType.getDatabaseTable();
 
-        for (CacheTypeMetadata type : types)
-            cfgs.add(from(type));
+                JdbcTypeField[] keyFields = jdbcType.getKeyFields();
+
+                meta.keyFields = new ArrayList<>(keyFields.length);
+
+                for (JdbcTypeField fld : keyFields)
+                    meta.keyFields.add(new VisorCacheTypeFieldMetadata(
+                        fld.getDatabaseFieldName(), fld.getDatabaseFieldType(),
+                        fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName())));
+
+                JdbcTypeField[] valFields = jdbcType.getValueFields();
+
+                meta.valFields = new ArrayList<>(valFields.length);
+
+                for (JdbcTypeField fld : valFields)
+                    meta.valFields.add(new VisorCacheTypeFieldMetadata(
+                            fld.getDatabaseFieldName(), fld.getDatabaseFieldType(),
+                            fld.getDatabaseFieldName(), U.compact(fld.getJavaFieldType().getName())));
+
+                if (notFound)
+                    metas.add(meta);
+            }
+        }
+
+        // Add old deprecated CacheTypeMetadata for compatibility.
+        if (types != null)
+            for (CacheTypeMetadata type : types)
+                metas.add(from(type));
+
+        return metas;
+    }
 
-        return cfgs;
+    /**
+     * @param q Actual cache query entities.
+     * @return Data transfer object for given cache type metadata.
+     */
+    public static VisorCacheTypeMetadata from(QueryEntity q) {
+        assert q != null;
+
+        VisorCacheTypeMetadata metadata = new VisorCacheTypeMetadata();
+
+        metadata.keyType = q.getKeyType();
+        metadata.valType = q.getValueType();
+
+        metadata.dbSchema = "";
+        metadata.dbTbl = "";
+
+        metadata.keyFields = Collections.emptyList();
+        metadata.valFields = Collections.emptyList();
+
+        LinkedHashMap<String, String> qryFields = q.getFields();
+
+        metadata.qryFlds = new LinkedHashMap<>(qryFields);
+
+        metadata.ascFlds = Collections.emptyMap();
+        metadata.descFlds = Collections.emptyMap();
+        metadata.txtFlds = Collections.emptyList();
+
+        Collection<QueryIndex> qryIdxs = q.getIndexes();
+
+        metadata.grps = new LinkedHashMap<>(qryIdxs.size());
+
+        for (QueryIndex qryIdx : qryIdxs) {
+            LinkedHashMap<String, Boolean> qryIdxFlds = qryIdx.getFields();
+
+            LinkedHashMap<String, IgniteBiTuple<String, Boolean>> grpFlds = new LinkedHashMap<>();
+
+            for (Map.Entry<String, Boolean> qryIdxFld : qryIdxFlds.entrySet()) {
+                String fldName = qryIdxFld.getKey();
+
+                grpFlds.put(fldName, new IgniteBiTuple<>(qryFields.get(fldName), !qryIdxFld.getValue()));
+            }
+
+            metadata.grps.put(qryIdx.getName(), grpFlds);
+        }
+
+        return metadata;
     }
 
     /**
@@ -246,4 +363,4 @@ public class VisorCacheTypeMetadata implements Serializable {
     public Map<String, LinkedHashMap<String, IgniteBiTuple<String, Boolean>>> grps() {
         return grps;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c42da8b/modules/core/src/main/resources/META-INF/classnames-jdk.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames-jdk.properties b/modules/core/src/main/resources/META-INF/classnames-jdk.properties
index 4dd20b7..bc74998 100644
--- a/modules/core/src/main/resources/META-INF/classnames-jdk.properties
+++ b/modules/core/src/main/resources/META-INF/classnames-jdk.properties
@@ -141,6 +141,8 @@ java.lang.reflect.MalformedParameterizedTypeException
 java.lang.reflect.Proxy
 java.lang.reflect.ReflectPermission
 java.lang.reflect.UndeclaredThrowableException
+java.math.BigDecimal
+java.math.BigInteger
 java.net.Authenticator$RequestorType
 java.net.BindException
 java.net.ConnectException


[34/43] ignite git commit: IGNITE-2007: IgniteBinary: metadata -> type.

Posted by yz...@apache.org.
IGNITE-2007: IgniteBinary: metadata -> type.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 0a97c1c10ed950ff8a6ef88c909e6f5ba84280d9
Parents: 5cf8dae
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Nov 26 12:41:52 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 26 12:41:52 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/IgniteBinary.java     | 10 +++++-----
 .../main/java/org/apache/ignite/binary/BinaryObject.java  |  2 +-
 .../main/java/org/apache/ignite/binary/BinaryType.java    |  2 +-
 .../processors/cache/portable/IgniteBinaryImpl.java       |  9 +++++----
 .../internal/processors/cacheobject/NoOpBinary.java       |  8 ++++----
 .../portable/BinaryObjectBuilderAdditionalSelfTest.java   |  6 +++---
 .../internal/portable/GridPortableMetaDataSelfTest.java   | 10 +++++-----
 ...dCacheClientNodeBinaryObjectMetadataMultinodeTest.java |  8 ++++----
 .../GridCacheClientNodeBinaryObjectMetadataTest.java      |  4 ++--
 .../platform/PlatformComputeBinarizableArgTask.java       |  2 +-
 10 files changed, 31 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java b/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
index 8802c10..1c63303 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteBinary.java
@@ -134,7 +134,7 @@ import org.jetbrains.annotations.Nullable;
  * <h1 class="header">Portable Metadata</h1>
  * Even though Ignite portable protocol only works with hash codes for type and field names
  * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgniteBinary#metadata(Class)}
+ * can be queried ar runtime via any of the {@link IgniteBinary#type(Class)}
  * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
  * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
  * <h1 class="header">Dynamic Structure Changes</h1>
@@ -323,7 +323,7 @@ public interface IgniteBinary {
      * @return Metadata.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    public BinaryType metadata(Class<?> cls) throws BinaryObjectException;
+    public BinaryType type(Class<?> cls) throws BinaryObjectException;
 
     /**
      * Gets metadata for provided class name.
@@ -332,7 +332,7 @@ public interface IgniteBinary {
      * @return Metadata.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    public BinaryType metadata(String typeName) throws BinaryObjectException;
+    public BinaryType type(String typeName) throws BinaryObjectException;
 
     /**
      * Gets metadata for provided type ID.
@@ -341,7 +341,7 @@ public interface IgniteBinary {
      * @return Metadata.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    public BinaryType metadata(int typeId) throws BinaryObjectException;
+    public BinaryType type(int typeId) throws BinaryObjectException;
 
     /**
      * Gets metadata for all known types.
@@ -349,5 +349,5 @@ public interface IgniteBinary {
      * @return Metadata.
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
-    public Collection<BinaryType> metadata() throws BinaryObjectException;
+    public Collection<BinaryType> types() throws BinaryObjectException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
index 432ace2..249daab 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
@@ -94,7 +94,7 @@ import java.util.TreeMap;
  * <h1 class="header">Binary Type Metadata</h1>
  * Even though Ignite binary protocol only works with hash codes for type and field names
  * to achieve better performance, Ignite provides metadata for all binary types which
- * can be queried ar runtime via any of the {@link org.apache.ignite.IgniteBinary#metadata(Class)}
+ * can be queried ar runtime via any of the {@link org.apache.ignite.IgniteBinary#type(Class)}
  * methods. Having metadata also allows for proper formatting of {@code BinaryObject.toString()} method,
  * even when binary objects are kept in binary format only, which may be necessary for audit reasons.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
index 710bf55..debd7e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
@@ -21,7 +21,7 @@ import java.util.Collection;
 
 /**
  * Binary type meta data. Metadata for binary types can be accessed from any of the
- * {@link org.apache.ignite.IgniteBinary#metadata(String)} methods.
+ * {@link org.apache.ignite.IgniteBinary#type(String)} methods.
  * Having metadata also allows for proper formatting of {@code BinaryObject#toString()} method,
  * even when binary objects are kept in binary format only, which may be necessary for audit reasons.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java
index 72f4d24..41aa68c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteBinaryImpl.java
@@ -59,6 +59,7 @@ public class IgniteBinaryImpl implements IgniteBinary {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public <T> T toBinary(@Nullable Object obj) throws BinaryObjectException {
         guard();
 
@@ -95,7 +96,7 @@ public class IgniteBinaryImpl implements IgniteBinary {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public BinaryType metadata(Class<?> cls) throws BinaryObjectException {
+    @Nullable @Override public BinaryType type(Class<?> cls) throws BinaryObjectException {
         guard();
 
         try {
@@ -107,7 +108,7 @@ public class IgniteBinaryImpl implements IgniteBinary {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public BinaryType metadata(String typeName) throws BinaryObjectException {
+    @Nullable @Override public BinaryType type(String typeName) throws BinaryObjectException {
         guard();
 
         try {
@@ -119,7 +120,7 @@ public class IgniteBinaryImpl implements IgniteBinary {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+    @Nullable @Override public BinaryType type(int typeId) throws BinaryObjectException {
         guard();
 
         try {
@@ -131,7 +132,7 @@ public class IgniteBinaryImpl implements IgniteBinary {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<BinaryType> metadata() throws BinaryObjectException {
+    @Override public Collection<BinaryType> types() throws BinaryObjectException {
         guard();
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/NoOpBinary.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/NoOpBinary.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/NoOpBinary.java
index 5bbc194..252baed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/NoOpBinary.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/NoOpBinary.java
@@ -49,22 +49,22 @@ public class NoOpBinary implements IgniteBinary {
     }
 
     /** {@inheritDoc} */
-    @Override public BinaryType metadata(Class<?> cls) throws BinaryObjectException {
+    @Override public BinaryType type(Class<?> cls) throws BinaryObjectException {
         throw unsupported();
     }
 
     /** {@inheritDoc} */
-    @Override public BinaryType metadata(String typeName) throws BinaryObjectException {
+    @Override public BinaryType type(String typeName) throws BinaryObjectException {
         throw unsupported();
     }
 
     /** {@inheritDoc} */
-    @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+    @Override public BinaryType type(int typeId) throws BinaryObjectException {
         throw unsupported();
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<BinaryType> metadata() throws BinaryObjectException {
+    @Override public Collection<BinaryType> types() throws BinaryObjectException {
         throw unsupported();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
index 356a25b..badad61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderAdditionalSelfTest.java
@@ -968,7 +968,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         mutableObj.build();
 
-        BinaryType metadata = portables().metadata(TestObjectContainer.class);
+        BinaryType metadata = portables().type(TestObjectContainer.class);
 
         assertEquals("String", metadata.fieldTypeName("xx567"));
     }
@@ -984,7 +984,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         mutableObj.build();
 
-        BinaryType metadata = portables().metadata(TestObjectContainer.class);
+        BinaryType metadata = portables().type(TestObjectContainer.class);
 
         assertEquals("String", metadata.fieldTypeName("xx567"));
     }
@@ -1008,7 +1008,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         mutableObj.build();
 
-        BinaryType metadata = portables().metadata(c.getClass());
+        BinaryType metadata = portables().type(c.getClass());
 
         assertTrue(metadata.fieldNames().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
             "colField", "mapField", "enumField", "enumArrField")));

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
index 5c25d3b..974b87e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
@@ -87,7 +87,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     public void testGetAll() throws Exception {
         portables().toBinary(new TestObject2());
 
-        Collection<BinaryType> metas = portables().metadata();
+        Collection<BinaryType> metas = portables().types();
 
         assertEquals(2, metas.size());
 
@@ -153,14 +153,14 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     public void testNoConfiguration() throws Exception {
         portables().toBinary(new TestObject3());
 
-        assertNotNull(portables().metadata(TestObject3.class));
+        assertNotNull(portables().type(TestObject3.class));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testReflection() throws Exception {
-        BinaryType meta = portables().metadata(TestObject1.class);
+        BinaryType meta = portables().type(TestObject1.class);
 
         assertNotNull(meta);
 
@@ -193,7 +193,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     public void testPortableMarshalAware() throws Exception {
         portables().toBinary(new TestObject2());
 
-        BinaryType meta = portables().metadata(TestObject2.class);
+        BinaryType meta = portables().type(TestObject2.class);
 
         assertNotNull(meta);
 
@@ -230,7 +230,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
         portables().toBinary(new TestObject2());
 
-        BinaryType meta = portables().metadata(TestObject2.class);
+        BinaryType meta = portables().type(TestObject2.class);
 
         assertNotNull(meta);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
index 99e2073..9733080 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java
@@ -158,7 +158,7 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
 
             IgniteBinary portables = ignite(i).binary();
 
-            Collection<BinaryType> metaCol = portables.metadata();
+            Collection<BinaryType> metaCol = portables.types();
 
             assertEquals(allTypes.size(), metaCol.size());
 
@@ -238,13 +238,13 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
 
             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                 @Override public boolean apply() {
-                    Collection<BinaryType> metaCol = p0.metadata();
+                    Collection<BinaryType> metaCol = p0.types();
 
                     return metaCol.size() == 1000;
                 }
             }, getTestTimeout());
 
-            Collection<BinaryType> metaCol = portables.metadata();
+            Collection<BinaryType> metaCol = portables.types();
 
             assertEquals(1000, metaCol.size());
 
@@ -290,6 +290,6 @@ public class GridCacheClientNodeBinaryObjectMetadataMultinodeTest extends GridCo
             cache.put(i, builder.build());
         }
 
-        assertEquals(100, ignite(0).binary().metadata().size());
+        assertEquals(100, ignite(0).binary().types().size());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
index 256e5fd..70d79d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeBinaryObjectMetadataTest.java
@@ -118,8 +118,8 @@ public class GridCacheClientNodeBinaryObjectMetadataTest extends GridCacheAbstra
                 aff0.mapKeyToPrimaryAndBackups(obj2));
         }
 
-        Collection<BinaryType> meta1 = ignite1.binary().metadata();
-        Collection<BinaryType> meta2 = ignite1.binary().metadata();
+        Collection<BinaryType> meta1 = ignite1.binary().types();
+        Collection<BinaryType> meta2 = ignite1.binary().types();
 
         assertEquals(meta1.size(), meta2.size());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a97c1c1/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
index ffae2d1..148d828 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputeBinarizableArgTask.java
@@ -89,7 +89,7 @@ public class PlatformComputeBinarizableArgTask extends ComputeTaskAdapter<Object
         @Nullable @Override public Object execute() {
             BinaryObjectEx arg0 = ((BinaryObjectEx)arg);
 
-            BinaryType meta = ignite.binary().metadata(arg0.typeId());
+            BinaryType meta = ignite.binary().type(arg0.typeId());
 
             if (meta == null)
                 throw new IgniteException("Metadata doesn't exist.");


[36/43] ignite git commit: IGNITE-2000 .Net: Fixed a bug causing exception in MessagingExample on second run when standalone node exists.

Posted by yz...@apache.org.
IGNITE-2000 .Net: Fixed a bug causing exception in MessagingExample on second run when standalone node exists.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: e386798fdf1c4d43afabc24e7206307958265c35
Parents: 446a14b
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Thu Nov 26 12:46:38 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Nov 26 12:46:38 2015 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs       | 5 ++++-
 .../Apache.Ignite.Examples/Messaging/MessagingExample.cs    | 9 +++++++--
 2 files changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e386798f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
index 087f886..bcf6baa 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
@@ -102,13 +102,16 @@ namespace Apache.Ignite.Core.Tests.Examples
                     if (example.NeedsTestDll)
                         args.Add(" -assembly=" + typeof(AverageSalaryJob).Assembly.Location);
 
-                    // ReSharper disable once UnusedVariable
                     var proc = new IgniteProcess(args.ToArray());
 
                     Assert.IsTrue(ignite.WaitTopology(i + 2, 30000));
+                    Assert.IsTrue(proc.Alive);
                 }
 
                 Ignition.ClientMode = clientMode;
+
+                // Run twice to catch issues with standalone node state
+                example.Run();
                 example.Run();
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e386798f/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
index ad28afe..3c74a42 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
@@ -72,13 +72,14 @@ namespace Apache.Ignite.Examples.Messaging
                     var unorderedCounter = new CountdownEvent(msgCount);
 
                     localMessaging.LocalListen(new LocalListener(unorderedCounter), Topic.Unordered);
+
                     localMessaging.LocalListen(new LocalListener(orderedCounter), Topic.Ordered);
 
                     // Set up remote listeners
                     var remoteMessaging = remotes.GetMessaging();
 
-                    remoteMessaging.RemoteListen(new RemoteUnorderedListener(), Topic.Unordered);
-                    remoteMessaging.RemoteListen(new RemoteOrderedListener(), Topic.Ordered);
+                    var idUnordered = remoteMessaging.RemoteListen(new RemoteUnorderedListener(), Topic.Unordered);
+                    var idOrdered = remoteMessaging.RemoteListen(new RemoteOrderedListener(), Topic.Ordered);
 
                     // Send unordered
                     Console.WriteLine(">>> Sending unordered messages...");
@@ -101,6 +102,10 @@ namespace Apache.Ignite.Examples.Messaging
 
                     unorderedCounter.Wait();
                     orderedCounter.Wait();
+
+                    // Unsubscribe
+                    remoteMessaging.StopRemoteListen(idUnordered);
+                    remoteMessaging.StopRemoteListen(idOrdered);
                 }
             }
 


[15/43] ignite git commit: ignite-1282 - Fixing tests.

Posted by yz...@apache.org.
ignite-1282 - Fixing tests.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 8e1d6c0e8fad981fc48dd0d149fb1abf30b1a7da
Parents: 385668b
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Nov 24 21:39:17 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Nov 24 21:39:17 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      | 11 ++++++++++
 .../cache/affinity/GridCacheAffinityImpl.java   |  3 ++-
 .../processors/query/GridQueryProcessor.java    | 22 +++++++++++++------
 .../GridPortableAffinityKeySelfTest.java        | 12 +++++++++-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java | 23 ++++++++++++++++++++
 5 files changed, 62 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8e1d6c0e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 1ff459e..765babc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -302,6 +302,17 @@ public class PortableContext implements Externalizable {
         for (TypeDescriptor desc : descs.descriptors()) {
             registerUserType(desc.clsName, desc.idMapper, desc.serializer, desc.affKeyFieldName);
         }
+
+        BinaryInternalIdMapper dfltMapper = BinaryInternalIdMapper.create(globalIdMapper);
+
+        // Put affinity field names for unconfigured types.
+        for (Map.Entry<String, String> entry : affFields.entrySet()) {
+            String typeName = entry.getKey();
+
+            int typeId = dfltMapper.typeId(typeName);
+
+            affKeyFieldNames.putIfAbsent(typeId, entry.getValue());
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e1d6c0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
index 33ab3f4..b9045e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
@@ -24,6 +24,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -150,7 +151,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
     @Override public Object affinityKey(K key) {
         A.notNull(key, "key");
 
-        if (key instanceof CacheObject)
+        if (key instanceof CacheObject && !(key instanceof BinaryObject))
             key = ((CacheObject)key).value(cctx.cacheObjectContext(), false);
 
         return cctx.config().getAffinityMapper().affinityKey(key);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e1d6c0e/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 d0eeeb1..c560e4e 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
@@ -1540,18 +1540,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String alias = aliases.get(fullName.toString());
 
-            ClassProperty tmp;
+            StringBuilder bld = new StringBuilder("get");
 
-            try {
-                StringBuilder bld = new StringBuilder("get");
+            bld.append(prop);
+
+            bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
 
-                bld.append(prop);
+            String[] mtdNames = new String[] {bld.toString(), prop};
 
-                bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
+            ClassProperty tmp = null;
 
-                tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias);
+            for (String mtdName : mtdNames) {
+                try {
+                    tmp = new ClassProperty(cls.getMethod(mtdName), key, alias);
+                }
+                catch (NoSuchMethodException ignore) {
+                    // No-op, will try another method or field.
+                }
             }
-            catch (NoSuchMethodException ignore) {
+
+            if (tmp == null) {
                 try {
                     tmp = new ClassProperty(cls.getDeclaredField(prop), key, alias);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e1d6c0e/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
index 3851bbc..c243901 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.configuration.BinaryConfiguration;
@@ -70,8 +71,9 @@ public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
         cfg.setBinaryConfiguration(bCfg);
 
         CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject.class.getName(), "affKey");
+        CacheKeyConfiguration keyCfg2 = new CacheKeyConfiguration("TestObject2", "affKey");
 
-        cfg.setCacheKeyCfg(keyCfg);
+        cfg.setCacheKeyCfg(keyCfg, keyCfg2);
 
         cfg.setMarshaller(new BinaryMarshaller());
 
@@ -135,6 +137,14 @@ public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
 
             assertEquals(i, aff.affinityKey(new TestObject(i)));
 
+            assertEquals(i, aff.affinityKey(ignite.binary().toBinary(new TestObject(i))));
+
+            BinaryObjectBuilder bldr = ignite.binary().builder("TestObject2");
+
+            bldr.setField("affKey", i);
+
+            assertEquals(i, aff.affinityKey(bldr.build()));
+
             CacheObject cacheObj = cacheObjProc.toCacheObject(cacheObjCtx, new TestObject(i), true);
 
             assertEquals(i, aff.affinityKey(cacheObj));

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e1d6c0e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index 9ccb893..1d21bdd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -809,6 +809,21 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testMethodAnnotationWithoutGet() throws Exception {
+        QueryCursor<List<?>> qry = grid(0).cache(null)
+            .query(new SqlFieldsQuery("select methodField from Organization where methodField='name-A'")
+            .setPageSize(10));
+
+        List<List<?>> flds = qry.getAll();
+
+        assertEquals(1, flds.size());
+
+        assertEquals("name-A", flds.get(0).get(0));
+    }
+
+    /**
      * @param cacheName Cache name.
      * @throws Exception If failed.
      */
@@ -995,6 +1010,14 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
             this.name = name;
         }
 
+        /**
+         * @return Generated method value.
+         */
+        @QuerySqlField
+        public String methodField() {
+            return "name-" + name;
+        }
+
         /** {@inheritDoc} */
         @Override public boolean equals(Object o) {
             if (this == o)


[37/43] ignite git commit: Fixed test to stop nodes after stop.

Posted by yz...@apache.org.
Fixed test to stop nodes after stop.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: be505f4846492cb37645349cd4d47ddc0f7e35e4
Parents: f2092e7
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 12:55:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 12:55:28 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/IgniteCacheSizeFailoverTest.java        | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be505f48/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
index 00b7c0f..1738a0d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
@@ -63,6 +63,13 @@ public class IgniteCacheSizeFailoverTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
     /**
      * @throws Exception If failed.
      */


[03/43] ignite git commit: Make binary marshaller default.

Posted by yz...@apache.org.
Make binary marshaller default.


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 3ca58d7b5edd0354c73450ff9fff3acf8a2da2cd
Parents: 7ea000a
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Nov 23 16:49:53 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Nov 23 16:49:53 2015 +0300

----------------------------------------------------------------------
 examples/config/example-ignite.xml              | 10 +--
 .../ignite/examples/binary/EmployeeKey.java     |  3 +
 .../configuration/CacheConfiguration.java       | 26 --------
 .../org/apache/ignite/internal/IgnitionEx.java  |  6 +-
 .../internal/portable/BinaryMarshaller.java     | 68 ++++++++++----------
 .../internal/portable/PortableContext.java      | 23 ++++++-
 6 files changed, 62 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca58d7b/examples/config/example-ignite.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-ignite.xml b/examples/config/example-ignite.xml
index d842a6d..a3e7e22 100644
--- a/examples/config/example-ignite.xml
+++ b/examples/config/example-ignite.xml
@@ -27,13 +27,5 @@
     <!-- Imports default Ignite configuration -->
     <import resource="example-default.xml"/>
 
-    <bean parent="ignite.cfg">
-        <!-- Enabled optimized marshaller -->
-        <property name="marshaller">
-            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
-                <!-- Set to false to allow non-serializable objects in examples, default is true. -->
-                <property name="requireSerializable" value="false"/>
-            </bean>
-        </property>
-    </bean>
+    <bean parent="ignite.cfg"/>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca58d7b/examples/src/main/java/org/apache/ignite/examples/binary/EmployeeKey.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/EmployeeKey.java b/examples/src/main/java/org/apache/ignite/examples/binary/EmployeeKey.java
index 2794230..7614f2c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/binary/EmployeeKey.java
+++ b/examples/src/main/java/org/apache/ignite/examples/binary/EmployeeKey.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.examples.binary;
 
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+
 /**
  * This class represents key for employee object.
  * <p>
@@ -28,6 +30,7 @@ public class EmployeeKey {
     private int id;
 
     /** Organization ID. */
+    @AffinityKeyMapped
     private int organizationId;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca58d7b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index e7dd9c3..8c3ea19 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -389,9 +389,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Query entities. */
     private Collection<QueryEntity> qryEntities;
 
-    /** Cache  */
-    private Collection<CacheKeyConfiguration> cacheKeyCfgs;
-
     /** Empty constructor (all values are initialized to their defaults). */
     public CacheConfiguration() {
         /* No-op. */
@@ -426,7 +423,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         atomicityMode = cc.getAtomicityMode();
         atomicWriteOrderMode = cc.getAtomicWriteOrderMode();
         backups = cc.getBackups();
-        cacheKeyCfgs = cc.getCacheKeyConfiguration();
         cacheLoaderFactory = cc.getCacheLoaderFactory();
         cacheMode = cc.getCacheMode();
         cacheWriterFactory = cc.getCacheWriterFactory();
@@ -997,28 +993,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Gets cache key configuration. Cache key configuration allows to override default cache key affinity
-     * by specifying affinity field name.
-     *
-     * @return Cache key configuration.
-     */
-    public Collection<CacheKeyConfiguration> getCacheKeyConfiguration() {
-        return cacheKeyCfgs;
-    }
-
-    /**
-     * Sets cache key configuration.
-     *
-     * @param cacheKeyCfgs Collection of cache key configurations.
-     */
-    public void setCacheKeyConfiguration(Collection<CacheKeyConfiguration> cacheKeyCfgs) {
-        if (this.cacheKeyCfgs != null)
-            this.cacheKeyCfgs.addAll(cacheKeyCfgs);
-        else
-            this.cacheKeyCfgs = cacheKeyCfgs;
-    }
-
-    /**
      * Gets cache atomicity mode.
      * <p>
      * Default value is defined by {@link #DFLT_CACHE_ATOMICITY_MODE}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca58d7b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 7d2b2dc..a83ddcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -61,6 +61,7 @@ import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.portable.BinaryMarshaller;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -76,7 +77,6 @@ import org.apache.ignite.logger.LoggerNodeIdAware;
 import org.apache.ignite.logger.java.JavaLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.mxbean.IgnitionMXBean;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
 import org.apache.ignite.resources.SpringApplicationContextResource;
@@ -1793,7 +1793,7 @@ public class IgnitionEx {
             Marshaller marsh = myCfg.getMarshaller();
 
             if (marsh == null) {
-                if (!OptimizedMarshaller.available()) {
+                if (!BinaryMarshaller.available()) {
                     U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
                         "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
                         "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +
@@ -1804,7 +1804,7 @@ public class IgnitionEx {
                     marsh = new JdkMarshaller();
                 }
                 else
-                    marsh = new OptimizedMarshaller();
+                    marsh = new BinaryMarshaller();
             }
 
             myCfg.setMarshaller(marsh);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca58d7b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
index 4f8c83c..759d802 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryMarshaller.java
@@ -24,54 +24,54 @@ import java.io.OutputStream;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.marshaller.AbstractMarshaller;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
 
 /**
  * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
- * in the portable format.
+ * in the binary format.
  * <p>
  * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
- * <p>
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This marshaller has no mandatory configuration parameters.
- * <h2 class="header">Java Example</h2>
- * <pre name="code" class="java">
- * PortableMarshaller marshaller = new PortableMarshaller();
- *
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * // Override marshaller.
- * cfg.setMarshaller(marshaller);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * PortableMarshaller can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
- *     ...
- *     &lt;property name="marshaller"&gt;
- *         &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *            ...
- *         &lt;/bean&gt;
- *     &lt;/property&gt;
- *     ...
- * &lt;/bean&gt;
- * </pre>
- * <p>
- * <img src="http://ignite.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
  */
 public class BinaryMarshaller extends AbstractMarshaller {
     /** */
     private GridPortableMarshaller impl;
 
     /**
+     * Checks whether {@code BinaryMarshaller} is able to work on the current JVM.
+     * <p>
+     * As long as {@code BinaryMarshaller} uses JVM-private API, which is not guaranteed
+     * to be available on all JVM, this method should be called to ensure marshaller could work properly.
+     * <p>
+     * Result of this method is automatically checked in constructor.
+     *
+     * @return {@code true} if {@code BinaryMarshaller} can work on the current JVM or
+     *      {@code false} if it can't.
+     */
+    @SuppressWarnings({"TypeParameterExtendsFinalClass", "ErrorNotRethrown"})
+    public static boolean available() {
+        try {
+            Unsafe unsafe = GridUnsafe.unsafe();
+
+            Class<? extends Unsafe> unsafeCls = unsafe.getClass();
+
+            unsafeCls.getMethod("allocateInstance", Class.class);
+            unsafeCls.getMethod("copyMemory", Object.class, long.class, Object.class, long.class, long.class);
+
+            return true;
+        }
+        catch (Exception ignored) {
+            return false;
+        }
+        catch (NoClassDefFoundError ignored) {
+            return false;
+        }
+    }
+
+    /**
      * Returns currently set {@link MarshallerContext}.
      *
      * @return Marshaller context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3ca58d7b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 8f37517..1ff459e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -24,6 +24,7 @@ import java.io.InvalidObjectException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.ObjectStreamException;
+import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.net.URISyntaxException;
 import java.net.URL;
@@ -52,6 +53,7 @@ import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
@@ -503,12 +505,14 @@ public class PortableContext implements Externalizable {
             throw new BinaryObjectException("Failed to register class.", e);
         }
 
+        String affFieldName = affinityFieldName(cls);
+
         PortableClassDescriptor desc = new PortableClassDescriptor(this,
             cls,
             true,
             typeId,
             typeName,
-            null,
+            affFieldName,
             idMapper,
             null,
             true,
@@ -518,7 +522,7 @@ public class PortableContext implements Externalizable {
 
         Collection<PortableSchema> schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null;
 
-        metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), null, schemas).wrap(this));
+        metaHnd.addMeta(typeId, new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, schemas).wrap(this));
 
         // perform put() instead of putIfAbsent() because "registered" flag might have been changed or class loader
         // might have reloaded described class.
@@ -606,6 +610,21 @@ public class PortableContext implements Externalizable {
         return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance();
     }
 
+    /**
+     * @param cls Class to get affinity field for.
+     * @return Affinity field name or {@code null} if field name was not found.
+     */
+    private String affinityFieldName(Class cls) {
+        for (; cls != Object.class; cls = cls.getSuperclass()) {
+            for (Field f : cls.getDeclaredFields()) {
+                if (f.getAnnotation(AffinityKeyMapped.class) != null)
+                    return f.getName();
+            }
+        }
+
+        return null;
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeString(out, igniteCfg.getGridName());


[31/43] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5

Posted by yz...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1.5


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: bfb839c035a7a7fc6c1a147647217893db456465
Parents: 30f01f5 5cf8dae
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 12:28:12 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 12:28:12 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/binary/BinaryObject.java  |   7 -
 .../org/apache/ignite/binary/BinaryType.java    |   7 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   3 +-
 .../internal/portable/BinaryFieldImpl.java      |   6 +-
 .../internal/portable/BinaryObjectEx.java       | 225 +----------------
 .../internal/portable/BinaryObjectExImpl.java   | 245 +++++++++++++++++++
 .../internal/portable/BinaryObjectImpl.java     |   2 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryTypeImpl.java       |   5 +
 .../ignite/internal/portable/PortableUtils.java |   2 +-
 .../builder/PortableBuilderSerializer.java      |   4 +-
 .../CacheObjectBinaryProcessorImpl.java         |   3 +-
 .../portable/BinaryFieldsAbstractSelfTest.java  |  12 +-
 .../portable/BinaryFieldsHeapSelfTest.java      |   2 +-
 .../portable/BinaryFieldsOffheapSelfTest.java   |   2 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   4 +-
 .../BinaryFooterOffsetsHeapSelfTest.java        |   2 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   2 +-
 .../portable/BinaryMarshallerSelfTest.java      |  16 +-
 .../portable/BinaryObjectBuilderSelfTest.java   |  71 +++---
 .../PlatformComputeBinarizableArgTask.java      |   6 +-
 21 files changed, 335 insertions(+), 293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsHeapSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFieldsOffheapSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsHeapSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryFooterOffsetsOffheapSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfb839c0/modules/core/src/test/java/org/apache/ignite/internal/portable/BinaryObjectBuilderSelfTest.java
----------------------------------------------------------------------


[27/43] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282

Posted by yz...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1282


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 7eea46d3ebb4268acf7e095187f5f2194732b89d
Parents: ae0ea3c d99fc8d
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Nov 26 11:39:19 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 26 11:39:19 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 .../client/router/RouterFactorySelfTest.java    |   2 +
 .../java/org/apache/ignite/IgniteCache.java     |  23 ++
 .../configuration/CacheConfiguration.java       |  30 +++
 .../internal/portable/BinaryReaderExImpl.java   |  18 +-
 .../processors/cache/GridCacheAdapter.java      | 156 ++++++++++++-
 .../processors/cache/GridCacheProxyImpl.java    |  61 +++++
 .../processors/cache/IgniteCacheProxy.java      |  42 ++++
 .../processors/cache/IgniteInternalCache.java   |  47 +++-
 .../distributed/dht/GridDhtCacheAdapter.java    |   7 +-
 .../distributed/near/GridNearCacheAdapter.java  |  10 +
 .../cache/query/GridCacheSqlQuery.java          |  42 ++++
 .../cache/query/GridCacheTwoStepQuery.java      |  35 ++-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../processors/query/GridQueryProcessor.java    |  40 +---
 .../processors/query/GridQueryProperty.java     |  45 ++++
 .../query/GridQueryTypeDescriptor.java          |   6 +
 .../visor/cache/VisorCacheConfiguration.java    |   4 +-
 .../cache/VisorCacheTypeFieldMetadata.java      |  30 ++-
 .../visor/cache/VisorCacheTypeMetadata.java     | 133 ++++++++++-
 .../ignite/resources/MBeanServerResource.java   |  69 ------
 .../META-INF/classnames-jdk.properties          |   2 +
 .../IgniteClientReconnectCacheTest.java         |   7 +
 .../CacheDeferredDeleteSanitySelfTest.java      |  98 ++++++++
 .../cache/CacheStopAndDestroySelfTest.java      |   8 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |   3 +
 .../IgniteCollectionAbstractTest.java           |   3 +
 .../IgniteCachePutRetryAbstractSelfTest.java    |   2 +
 .../multijvm/IgniteCacheProcessProxy.java       |  38 ++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../query/h2/GridH2ResultSetIterator.java       |  19 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 227 ++++++++++++++++---
 .../query/h2/opt/GridH2RowDescriptor.java       |   5 +
 .../processors/query/h2/opt/GridH2Table.java    |  69 +++++-
 .../query/h2/opt/GridH2TreeIndex.java           |  98 +++++---
 .../query/h2/sql/GridSqlQuerySplitter.java      |  73 ++++--
 .../processors/query/h2/sql/GridSqlSelect.java  |  27 +++
 .../query/h2/twostep/GridMapQueryExecutor.java  |  17 +-
 .../query/h2/twostep/GridMergeIndex.java        |   7 +
 .../h2/twostep/GridMergeIndexUnsorted.java      |  14 ++
 .../h2/twostep/GridReduceQueryExecutor.java     | 117 +++++++---
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  69 +++---
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   7 +-
 ...PartitionedSnapshotEnabledQuerySelfTest.java |  28 +++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  19 ++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 .../Services/ServicesExample.cs                 |   4 +
 modules/scalar/pom.xml                          |   4 +-
 modules/spark/pom.xml                           |   4 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../ignite-int-max-values-offheap-config.xml    |  89 ++++++++
 .../ignite-int-max-values-onheap-config.xml     |  84 +++++++
 .../ignite-int-max-values-swap-config.xml       |  93 ++++++++
 .../test-max-int-values-offheap.properties      |  70 ++++++
 .../test-max-int-values-onheap.properties       |  70 ++++++
 .../config/test-max-int-values-swap.properties  |  69 ++++++
 .../yardstick/cache/IntMaxValueEntriesTest.java | 135 +++++++++++
 parent/pom.xml                                  |   2 +-
 58 files changed, 2071 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/7eea46d3/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------