You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2015/06/18 04:25:26 UTC

[01/17] incubator-ignite git commit: ignite-1020 create GridDhtTxLocal with storeEnabled=true

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-884 edc42c427 -> 5a3c88ada


ignite-1020 create GridDhtTxLocal with storeEnabled=true


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

Branch: refs/heads/ignite-884
Commit: 7fd14411c36b6de7b79841c9af81506780279a10
Parents: e3750b7
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 16 17:14:54 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 16 17:14:54 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtTransactionalCacheAdapter.java    | 2 +-
 .../internal/processors/cache/transactions/IgniteTxHandler.java    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7fd14411/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 703daf9..4f081bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -859,7 +859,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                         req.isolation(),
                                         req.timeout(),
                                         req.isInvalidate(),
-                                        false,
+                                        true,
                                         req.txSize(),
                                         null,
                                         req.subjectId(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7fd14411/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 01662ef..e481e25 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -325,7 +325,7 @@ public class IgniteTxHandler {
                     req.isolation(),
                     req.timeout(),
                     req.isInvalidate(),
-                    false,
+                    true,
                     req.txSize(),
                     req.transactionNodes(),
                     req.subjectId(),


[15/17] incubator-ignite git commit: Merge branches 'ignite-884' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-884

Posted by vk...@apache.org.
Merge branches 'ignite-884' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-884


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

Branch: refs/heads/ignite-884
Commit: e0110110d7d68860cede0e5161c1a9259e825628
Parents: edc42c4 af829d0
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 17 18:43:20 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 17 18:43:20 2015 -0700

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  13 +-
 .../client/GridClientConfiguration.java         |   2 +-
 .../GridClientOptimizedMarshaller.java          |  21 ++
 .../impl/GridTcpRouterNioListenerAdapter.java   |   2 +-
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../processors/cache/IgniteCacheProxy.java      |   7 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   7 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |   3 +-
 .../ignite/internal/util/IgniteUtils.java       |  21 ++
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 +++++++++++++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++++++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 ++
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 ++
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++++++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 ++
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 57 files changed, 875 insertions(+), 59 deletions(-)
----------------------------------------------------------------------



[12/17] incubator-ignite git commit: Fixed tests.

Posted by vk...@apache.org.
Fixed tests.


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

Branch: refs/heads/ignite-884
Commit: 5218210cf3400913ab2116684b23503efbbd32a3
Parents: 9c0b09c
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 14:36:44 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 14:36:44 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/client/GridClientConfiguration.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5218210c/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
index f06d779..7fa0dae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
@@ -110,7 +110,7 @@ public class GridClientConfiguration {
     private ExecutorService executor;
 
     /** Marshaller. */
-    private GridClientMarshaller marshaller;
+    private GridClientMarshaller marshaller = new GridClientOptimizedMarshaller(U.allPluginProviders());
 
     /** Daemon flag. */
     private boolean daemon;
@@ -119,7 +119,7 @@ public class GridClientConfiguration {
      * Creates default configuration.
      */
     public GridClientConfiguration() {
-        marshaller = new GridClientOptimizedMarshaller(U.allPluginProviders());
+        // No-op.
     }
 
     /**


[14/17] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1020' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1020' into ignite-sprint-6


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

Branch: refs/heads/ignite-884
Commit: af829d0ee0a646ac7ed96c65ddda7fc6f38212b0
Parents: 5218210 674072a
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 17 17:19:58 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 17 17:19:58 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../processors/cache/IgniteCacheProxy.java      |   7 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   7 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 +++++++++++++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++++++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 ++
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 ++
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++++++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 ++
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 14 files changed, 791 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[05/17] incubator-ignite git commit: #IGNITE-GG-10429 Fixed client marshaller.

Posted by vk...@apache.org.
#IGNITE-GG-10429 Fixed client marshaller.


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

Branch: refs/heads/ignite-884
Commit: 9142eb7ce1cb0f665c129580a3a611a6ad5dce1e
Parents: 092bb91
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 10:30:04 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 10:30:04 2015 +0300

----------------------------------------------------------------------
 .../impl/GridTcpRouterNioListenerAdapter.java      | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9142eb7c/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
index 0bf416c..e60f2aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
@@ -25,8 +25,10 @@ import org.apache.ignite.internal.client.marshaller.optimized.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.plugin.*;
 import org.jetbrains.annotations.*;
 
+import java.security.*;
 import java.util.*;
 
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.*;
@@ -65,7 +67,20 @@ public abstract class GridTcpRouterNioListenerAdapter implements GridNioServerLi
 
         marshMap = new HashMap<>();
 
-        marshMap.put(GridClientOptimizedMarshaller.ID, new GridClientOptimizedMarshaller());
+        List<PluginProvider> plugins = AccessController.doPrivileged(new PrivilegedAction<List<PluginProvider>>() {
+            @Override public List<PluginProvider> run() {
+                List<PluginProvider> providers = new ArrayList<>();
+
+                ServiceLoader<PluginProvider> ldr = ServiceLoader.load(PluginProvider.class);
+
+                for (PluginProvider provider : ldr)
+                    providers.add(provider);
+
+                return providers;
+            }
+        });
+
+        marshMap.put(GridClientOptimizedMarshaller.ID, new GridClientOptimizedMarshaller(plugins));
         marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller());
 
         init();


[09/17] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-1020

Posted by vk...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-1020


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

Branch: refs/heads/ignite-884
Commit: e507f030998c98851f97affa14667b64ebdb236e
Parents: a75b868 01eee2d
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 17 12:10:41 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 17 12:10:41 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 13 +-----------
 .../client/GridClientConfiguration.java         |  4 ++--
 .../GridClientOptimizedMarshaller.java          | 21 ++++++++++++++++++++
 .../impl/GridTcpRouterNioListenerAdapter.java   |  2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  3 ++-
 .../ignite/internal/util/IgniteUtils.java       | 21 ++++++++++++++++++++
 6 files changed, 48 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[08/17] incubator-ignite git commit: # ignite-1020 properly skip non-primary entries for store update , tests

Posted by vk...@apache.org.
# ignite-1020 properly skip non-primary entries for store update , tests


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

Branch: refs/heads/ignite-884
Commit: a75b868605f661a71ed96f17e9d36471ccbfcd13
Parents: 9a0e659
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 17 11:25:23 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 17 12:07:37 2015 +0300

----------------------------------------------------------------------
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 +++++++++++++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++++++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 ++
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 ++
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++++++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 ++
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 9 files changed, 772 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/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 bc6308b..7bc9042 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
@@ -500,12 +500,16 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     List<Object> rmvCol = null;
                     CacheStoreManager writeStore = null;
 
-                    boolean skipNear = near() && isWriteToStoreFromDht;
+                    boolean skipNonPrimary = near() && isWriteToStoreFromDht;
 
                     for (IgniteTxEntry e : writeEntries) {
-                        boolean skip = (skipNear && e.cached().isNear()) ||
-                            e.skipStore() ||
-                            (e.context().store().isLocal() && !e.context().affinityNode());
+                        boolean skip = e.skipStore();
+
+                        if (!skip && skipNonPrimary) {
+                            skip = e.cached().isNear() ||
+                                e.cached().detached() ||
+                                !e.context().affinity().primary(e.cached().partition(), topologyVersion()).isLocal();
+                        }
 
                         if (skip)
                             continue;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeAbstractTest.java
new file mode 100644
index 0000000..79a0293
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeAbstractTest.java
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.store.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.internal.IgniteNodeAttributes.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public abstract class CacheStoreUsageMultinodeAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected boolean client;
+
+    /** */
+    protected boolean cache;
+
+    /** */
+    protected boolean cacheStore;
+
+    /** */
+    protected boolean locStore;
+
+    /** */
+    protected boolean writeBehind;
+
+    /** */
+    protected boolean nearCache;
+
+    /** */
+    protected static Map<String, List<Cache.Entry<?, ?>>> writeMap;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClientMode(client);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        if (cache)
+            cfg.setCacheConfiguration(cacheConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setAtomicityMode(atomicityMode());
+        ccfg.setBackups(1);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        if (cacheStore) {
+            if (writeBehind) {
+                ccfg.setWriteBehindEnabled(true);
+                ccfg.setWriteBehindFlushFrequency(100);
+            }
+
+            ccfg.setWriteThrough(true);
+
+            ccfg.setCacheStoreFactory(locStore ? new TestLocalStoreFactory() : new TestStoreFactory());
+        }
+
+        if (nearCache)
+            ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        return ccfg;
+    }
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        writeMap = new HashMap<>();
+    }
+
+    /**
+     * @param clientStore {@code True} if store configured on client node.
+     * @throws Exception If failed.
+     */
+    protected void checkStoreUpdate(boolean clientStore) throws Exception {
+        Ignite client = grid(3);
+
+        assertTrue(client.configuration().isClientMode());
+
+        awaitPartitionMapExchange();
+
+        IgniteCache<Object, Object> cache0 = ignite(0).cache(null);
+        IgniteCache<Object, Object> cache1 = ignite(1).cache(null);
+        IgniteCache<Object, Object> clientCache = client.cache(null);
+
+        assertTrue(((IgniteCacheProxy)cache0).context().store().configured());
+        assertEquals(clientStore, ((IgniteCacheProxy) clientCache).context().store().configured());
+
+        List<TransactionConcurrency> tcList = new ArrayList<>();
+
+        tcList.add(null);
+
+        if (atomicityMode() == TRANSACTIONAL) {
+            tcList.add(TransactionConcurrency.OPTIMISTIC);
+            tcList.add(TransactionConcurrency.PESSIMISTIC);
+        }
+
+        log.info("Start test [atomicityMode=" + atomicityMode() +
+            ", locStore=" + locStore +
+            ", writeBehind=" + writeBehind +
+            ", nearCache=" + nearCache +
+            ", clientStore=" + clientStore + ']');
+
+        for (TransactionConcurrency tc : tcList) {
+            testStoreUpdate(cache0, primaryKey(cache0), tc);
+
+            testStoreUpdate(cache0, backupKey(cache0), tc);
+
+            testStoreUpdate(cache0, nearKey(cache0), tc);
+
+            testStoreUpdate(cache0, primaryKey(cache1), tc);
+
+            testStoreUpdate(clientCache, primaryKey(cache0), tc);
+
+            testStoreUpdate(clientCache, primaryKey(cache1), tc);
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     * @param tc Transaction concurrency mode.
+     * @throws Exception If failed.
+     */
+    protected void testStoreUpdate(IgniteCache<Object, Object> cache,
+       Object key,
+       @Nullable TransactionConcurrency tc)
+        throws Exception
+    {
+        boolean storeOnPrimary = atomicityMode() == ATOMIC || locStore || writeBehind;
+
+        assertTrue(writeMap.isEmpty());
+
+        Ignite ignite = cache.unwrap(Ignite.class);
+
+        Affinity<Object> obj = ignite.affinity(cache.getName());
+
+        ClusterNode node = obj.mapKeyToNode(key);
+
+        assertNotNull(node);
+
+        String expNode = storeOnPrimary ? (String)node.attribute(ATTR_GRID_NAME) : ignite.name();
+
+        assertNotNull(expNode);
+
+        log.info("Put [node=" + ignite.name() +
+            ", key=" + key +
+            ", primary=" + node.attribute(ATTR_GRID_NAME) +
+            ", tx=" + tc +
+            ", nearCache=" + (cache.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null) +
+            ", storeOnPrimary=" + storeOnPrimary + ']');
+
+        Transaction tx = tc != null ? ignite.transactions().txStart(tc, REPEATABLE_READ) : null;
+
+        cache.put(key, key);
+
+        if (tx != null)
+            tx.commit();
+
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override
+            public boolean apply() {
+                return writeMap.size() > 0;
+            }
+        }, 1000);
+
+        assertTrue("Store is not updated", wait);
+
+        assertEquals("Write on wrong node: " + writeMap, 1, writeMap.size());
+
+        assertEquals(expNode, writeMap.keySet().iterator().next());
+
+        writeMap.clear();
+    }
+
+    /**
+     *
+     */
+    public static class TestStoreFactory implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return new TestStore();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class TestLocalStoreFactory implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return new TestLocalStore();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class TestStore extends CacheStoreAdapter<Object, Object> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("SynchronizeOnNonFinalField")
+        @Override public void write(Cache.Entry<?, ?> entry) {
+            synchronized (writeMap) {
+                ignite.log().info("Write [node=" + ignite.name() + ", entry=" + entry + ']');
+
+                String name = ignite.name();
+
+                List<Cache.Entry<?, ?>> list = writeMap.get(name);
+
+                if (list == null) {
+                    list = new ArrayList<>();
+
+                    writeMap.put(name, list);
+                }
+
+                list.add(entry);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object load(Object key) throws CacheLoaderException {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+    /**
+     *
+     */
+    @CacheLocalStore
+    public static class TestLocalStore extends TestStore {
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
new file mode 100644
index 0000000..82667d9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAbstractTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ *
+ */
+public abstract class CacheStoreUsageMultinodeDynamicStartAbstractTest extends CacheStoreUsageMultinodeAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        cache = false;
+
+        startGridsMultiThreaded(3);
+
+        client = true;
+
+        startGrid(3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStart() throws Exception {
+        checkStoreWithDynamicStart(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartNearEnabled() throws Exception {
+        nearCache = true;
+
+        checkStoreWithDynamicStart(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicFromClientStart() throws Exception {
+        checkStoreWithDynamicStart(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartFromClientNearEnabled() throws Exception {
+        nearCache = true;
+
+        checkStoreWithDynamicStart(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartLocalStore() throws Exception {
+        locStore = true;
+
+        checkStoreWithDynamicStart(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartFromClientLocalStore() throws Exception {
+        locStore = true;
+
+        checkStoreWithDynamicStart(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartLocalStoreNearEnabled() throws Exception {
+        locStore = true;
+
+        nearCache = true;
+
+        checkStoreWithDynamicStart(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartWriteBehindStore() throws Exception {
+        writeBehind = true;
+
+        checkStoreWithDynamicStart(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartFromClientWriteBehindStore() throws Exception {
+        writeBehind = true;
+
+        checkStoreWithDynamicStart(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDynamicStartWriteBehindStoreNearEnabled() throws Exception {
+        writeBehind = true;
+
+        nearCache = true;
+
+        checkStoreWithDynamicStart(false);
+    }
+
+    /**
+     * @param clientStart {@code True} if start cache from client node.
+     * @throws Exception If failed.
+     */
+    private void checkStoreWithDynamicStart(boolean clientStart) throws Exception {
+        cacheStore = true;
+
+        CacheConfiguration ccfg = cacheConfiguration();
+
+        assertNotNull(ccfg.getCacheStoreFactory());
+
+        Ignite srv = ignite(0);
+
+        Ignite client = ignite(3);
+
+        Ignite node = clientStart ? client : srv;
+
+        IgniteCache cache = nearCache ? node.createCache(ccfg, new NearCacheConfiguration()) : node.createCache(ccfg);
+
+        assertNotNull(cache);
+
+        try {
+            if (nearCache)
+                client.createNearCache(null, new NearCacheConfiguration<>());
+
+            checkStoreUpdate(true);
+        }
+        finally {
+            cache = srv.cache(null);
+
+            if (cache != null)
+                cache.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAtomicTest.java
new file mode 100644
index 0000000..884dedd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartAtomicTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class CacheStoreUsageMultinodeDynamicStartAtomicTest extends CacheStoreUsageMultinodeDynamicStartAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartTxTest.java
new file mode 100644
index 0000000..825b450
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeDynamicStartTxTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class CacheStoreUsageMultinodeDynamicStartTxTest extends CacheStoreUsageMultinodeDynamicStartAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAbstractTest.java
new file mode 100644
index 0000000..c00d003
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAbstractTest.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.transactions.*;
+
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public abstract class CacheStoreUsageMultinodeStaticStartAbstractTest extends CacheStoreUsageMultinodeAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfiguration() throws Exception {
+        checkStoreUpdateStaticConfig(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationNearEnabled() throws Exception {
+        nearCache = true;
+
+        checkStoreUpdateStaticConfig(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationLocalStore() throws Exception {
+        locStore = true;
+
+        checkStoreUpdateStaticConfig(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationLocalStoreNearEnabled() throws Exception {
+        locStore = true;
+
+        nearCache = true;
+
+        checkStoreUpdateStaticConfig(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationTxLocalStoreNoClientStore() throws Exception {
+        locStore = true;
+
+        checkStoreUpdateStaticConfig(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationTxLocalStoreNoClientStoreNearEnabled() throws Exception {
+        locStore = true;
+
+        nearCache = true;
+
+        checkStoreUpdateStaticConfig(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationTxWriteBehindStore() throws Exception {
+        writeBehind = true;
+
+        checkStoreUpdateStaticConfig(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationTxWriteBehindStoreNearEnabled() throws Exception {
+        writeBehind = true;
+
+        nearCache = true;
+
+        checkStoreUpdateStaticConfig(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationTxWriteBehindStoreNoClientStore() throws Exception {
+        writeBehind = true;
+
+        checkStoreUpdateStaticConfig(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStaticConfigurationTxWriteBehindStoreNoClientStoreNearEnabled() throws Exception {
+        writeBehind = true;
+
+        nearCache = true;
+
+        checkStoreUpdateStaticConfig(false);
+    }
+
+    /**
+     * @param clientStore {@code True} if store should be configured on client node.
+     * @throws Exception If failed.
+     */
+    private void checkStoreUpdateStaticConfig(boolean clientStore) throws Exception {
+        try {
+            cache = true;
+
+            cacheStore = true;
+
+            startGridsMultiThreaded(3);
+
+            client = true;
+
+            cacheStore = clientStore;
+
+            startGrid(3);
+
+            checkStoreUpdate(clientStore);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAtomicTest.java
new file mode 100644
index 0000000..6908aaa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartAtomicTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class CacheStoreUsageMultinodeStaticStartAtomicTest extends CacheStoreUsageMultinodeStaticStartAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartTxTest.java
new file mode 100644
index 0000000..b42fd44
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheStoreUsageMultinodeStaticStartTxTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class CacheStoreUsageMultinodeStaticStartTxTest extends CacheStoreUsageMultinodeStaticStartAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a75b8686/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index d155330..6295a4d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -135,6 +135,10 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class);
 
         suite.addTestSuite(CacheClientStoreSelfTest.class);
+        suite.addTestSuite(CacheStoreUsageMultinodeStaticStartAtomicTest.class);
+        suite.addTestSuite(CacheStoreUsageMultinodeStaticStartTxTest.class);
+        suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartAtomicTest.class);
+        suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartTxTest.class);
 
         suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
         suite.addTestSuite(IgniteStartCacheInTransactionAtomicSelfTest.class);


[06/17] incubator-ignite git commit: #IGNITE-GG-10429 Extract utility method.

Posted by vk...@apache.org.
#IGNITE-GG-10429 Extract utility method.


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

Branch: refs/heads/ignite-884
Commit: 01eee2da37961d8f6c8ae477f4a6b3cb8a6a6372
Parents: 9142eb7
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 10:48:19 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 10:48:19 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 13 +-----------
 .../client/GridClientConfiguration.java         | 17 +---------------
 .../impl/GridTcpRouterNioListenerAdapter.java   | 17 +---------------
 .../ignite/internal/util/IgniteUtils.java       | 21 ++++++++++++++++++++
 4 files changed, 24 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01eee2da/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 9e0dcbf..e19d3d3 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
@@ -630,18 +630,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         // Ack configuration.
         ackSpis();
 
-        List<PluginProvider> plugins = AccessController.doPrivileged(new PrivilegedAction<List<PluginProvider>>() {
-            @Override public List<PluginProvider> run() {
-                List<PluginProvider> providers = new ArrayList<>();
-
-                ServiceLoader<PluginProvider> ldr = ServiceLoader.load(PluginProvider.class);
-
-                for (PluginProvider provider : ldr)
-                    providers.add(provider);
-
-                return providers;
-            }
-        });
+        List<PluginProvider> plugins = U.allPluginProviders();
 
         // Spin out SPIs & managers.
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01eee2da/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
index 4a909a3..f06d779 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
@@ -23,12 +23,10 @@ import org.apache.ignite.internal.client.marshaller.optimized.*;
 import org.apache.ignite.internal.client.ssl.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.plugin.*;
 import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import java.net.*;
-import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 
@@ -121,20 +119,7 @@ public class GridClientConfiguration {
      * Creates default configuration.
      */
     public GridClientConfiguration() {
-        List<PluginProvider> plugins = AccessController.doPrivileged(new PrivilegedAction<List<PluginProvider>>() {
-            @Override public List<PluginProvider> run() {
-                List<PluginProvider> providers = new ArrayList<>();
-
-                ServiceLoader<PluginProvider> ldr = ServiceLoader.load(PluginProvider.class);
-
-                for (PluginProvider provider : ldr)
-                    providers.add(provider);
-
-                return providers;
-            }
-        });
-
-        marshaller = new GridClientOptimizedMarshaller(plugins);
+        marshaller = new GridClientOptimizedMarshaller(U.allPluginProviders());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01eee2da/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
index e60f2aa..1775b21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java
@@ -25,10 +25,8 @@ import org.apache.ignite.internal.client.marshaller.optimized.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.plugin.*;
 import org.jetbrains.annotations.*;
 
-import java.security.*;
 import java.util.*;
 
 import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.*;
@@ -67,20 +65,7 @@ public abstract class GridTcpRouterNioListenerAdapter implements GridNioServerLi
 
         marshMap = new HashMap<>();
 
-        List<PluginProvider> plugins = AccessController.doPrivileged(new PrivilegedAction<List<PluginProvider>>() {
-            @Override public List<PluginProvider> run() {
-                List<PluginProvider> providers = new ArrayList<>();
-
-                ServiceLoader<PluginProvider> ldr = ServiceLoader.load(PluginProvider.class);
-
-                for (PluginProvider provider : ldr)
-                    providers.add(provider);
-
-                return providers;
-            }
-        });
-
-        marshMap.put(GridClientOptimizedMarshaller.ID, new GridClientOptimizedMarshaller(plugins));
+        marshMap.put(GridClientOptimizedMarshaller.ID, new GridClientOptimizedMarshaller(U.allPluginProviders()));
         marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller());
 
         init();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/01eee2da/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index a303abc..6623e85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.plugin.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.discovery.*;
@@ -629,6 +630,26 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Gets all plugin providers.
+     *
+     * @return Plugins.
+     */
+    public static List<PluginProvider> allPluginProviders() {
+        return AccessController.doPrivileged(new PrivilegedAction<List<PluginProvider>>() {
+            @Override public List<PluginProvider> run() {
+                List<PluginProvider> providers = new ArrayList<>();
+
+                ServiceLoader<PluginProvider> ldr = ServiceLoader.load(PluginProvider.class);
+
+                for (PluginProvider provider : ldr)
+                    providers.add(provider);
+
+                return providers;
+            }
+        });
+    }
+
+    /**
      * Converts exception, but unlike {@link #convertException(IgniteCheckedException)}
      * does not wrap passed in exception if none suitable converter found.
      *


[11/17] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/ignite-884
Commit: 9c0b09c58965091e866d0ebdba8055d5471c6e9a
Parents: f173f91 dc05e88
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 12:34:04 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 12:34:04 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                  | 2 +-
 modules/aop/pom.xml                               | 2 +-
 modules/aws/pom.xml                               | 2 +-
 modules/clients/pom.xml                           | 2 +-
 modules/cloud/pom.xml                             | 2 +-
 modules/codegen/pom.xml                           | 2 +-
 modules/core/pom.xml                              | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 modules/extdata/p2p/pom.xml                       | 2 +-
 modules/extdata/uri/pom.xml                       | 2 +-
 modules/gce/pom.xml                               | 2 +-
 modules/geospatial/pom.xml                        | 2 +-
 modules/hadoop/pom.xml                            | 2 +-
 modules/hibernate/pom.xml                         | 2 +-
 modules/indexing/pom.xml                          | 2 +-
 modules/jcl/pom.xml                               | 2 +-
 modules/jta/pom.xml                               | 2 +-
 modules/log4j/pom.xml                             | 2 +-
 modules/mesos/pom.xml                             | 2 +-
 modules/rest-http/pom.xml                         | 2 +-
 modules/scalar-2.10/pom.xml                       | 2 +-
 modules/scalar/pom.xml                            | 2 +-
 modules/schedule/pom.xml                          | 2 +-
 modules/schema-import/pom.xml                     | 2 +-
 modules/slf4j/pom.xml                             | 2 +-
 modules/spark-2.10/pom.xml                        | 2 +-
 modules/spark/pom.xml                             | 2 +-
 modules/spring/pom.xml                            | 2 +-
 modules/ssh/pom.xml                               | 2 +-
 modules/tools/pom.xml                             | 2 +-
 modules/urideploy/pom.xml                         | 2 +-
 modules/visor-console-2.10/pom.xml                | 2 +-
 modules/visor-console/pom.xml                     | 2 +-
 modules/visor-plugins/pom.xml                     | 2 +-
 modules/web/pom.xml                               | 2 +-
 modules/yardstick/pom.xml                         | 2 +-
 pom.xml                                           | 2 +-
 37 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[17/17] incubator-ignite git commit: IGNITE-884 - Spring autowiring

Posted by vk...@apache.org.
IGNITE-884 - Spring autowiring


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

Branch: refs/heads/ignite-884
Commit: 5a3c88ada0bcf2f2835fc6a1bcff7d8a27851437
Parents: aaa7818
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 17 19:10:49 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 17 19:10:49 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5a3c88ad/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
index 0c2e99e..3025208 100644
--- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
+++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.store.spring.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.p2p.*;
 import org.apache.ignite.spring.*;
+import org.apache.ignite.spring.autowired.*;
 
 /**
  * Spring tests.
@@ -55,6 +56,8 @@ public class IgniteSpringTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheJdbcPojoStoreFactorySelfTest.class);
 
+        suite.addTestSuite(SpringAutowiredSelfTest.class);
+
         return suite;
     }
 }


[13/17] incubator-ignite git commit: # ignite-1020 do not write to store for local store load

Posted by vk...@apache.org.
# ignite-1020 do not write to store for local store load


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

Branch: refs/heads/ignite-884
Commit: 674072a336a2aa276373a5f6820a9e66df88ede8
Parents: e507f03
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 17 15:31:57 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 17 15:31:57 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheAdapter.java    | 4 ++++
 .../ignite/internal/processors/cache/IgniteCacheProxy.java    | 7 +++++++
 .../processors/dr/IgniteDrDataStreamerCacheUpdater.java       | 7 ++++++-
 3 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/674072a3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2ca7687..7335d72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3305,6 +3305,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 DataStreamerImpl ldr = ctx.kernalContext().dataStream().dataStreamer(ctx.namex());
 
                 try {
+                    ldr.skipStore(true);
+
                     ldr.receiver(new IgniteDrDataStreamerCacheUpdater());
 
                     LocalStoreLoadClosure c = new LocalStoreLoadClosure(p, ldr, plc);
@@ -3504,6 +3506,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             DataStreamerImpl ldr = ctx.kernalContext().dataStream().dataStreamer(ctx.namex());
 
             try {
+                ldr.skipStore(true);
+
                 ldr.receiver(new IgniteDrDataStreamerCacheUpdater());
 
                 LocalStoreLoadClosure c = new LocalStoreLoadClosure(null, ldr, plc0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/674072a3/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 69ce7b0..0d70792 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
@@ -139,6 +139,13 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /**
+     * @return Operation context.
+     */
+    @Nullable public CacheOperationContext operationContext() {
+        return opCtx;
+    }
+
+    /**
      * Gets cache proxy which does not acquire read lock on gateway enter, should be
      * used only if grid read lock is externally acquired.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/674072a3/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/IgniteDrDataStreamerCacheUpdater.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/IgniteDrDataStreamerCacheUpdater.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/IgniteDrDataStreamerCacheUpdater.java
index c3a8989..e5bbe39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/IgniteDrDataStreamerCacheUpdater.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/IgniteDrDataStreamerCacheUpdater.java
@@ -46,7 +46,12 @@ public class IgniteDrDataStreamerCacheUpdater implements StreamReceiver<KeyCache
 
             GridKernalContext ctx = ((IgniteKernal)cache0.unwrap(Ignite.class)).context();
             IgniteLogger log = ctx.log(IgniteDrDataStreamerCacheUpdater.class);
-            GridCacheAdapter cache = ctx.cache().internalCache(cacheName);
+            GridCacheAdapter internalCache = ctx.cache().internalCache(cacheName);
+
+            CacheOperationContext opCtx = ((IgniteCacheProxy)cache0).operationContext();
+
+            IgniteInternalCache cache =
+                opCtx != null ? new GridCacheProxyImpl(internalCache.context(), internalCache, opCtx) : internalCache;
 
             assert !F.isEmpty(col);
 


[10/17] incubator-ignite git commit: Merge branch 'ignite-gg-10429' into ignite-sprint-6

Posted by vk...@apache.org.
Merge branch 'ignite-gg-10429' into ignite-sprint-6


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

Branch: refs/heads/ignite-884
Commit: f173f9132eac1b7f8ac31bad795f0694be9a3c14
Parents: d1a2150 01eee2d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 12:33:13 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 12:33:13 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 13 +-----------
 .../client/GridClientConfiguration.java         |  4 ++--
 .../GridClientOptimizedMarshaller.java          | 21 ++++++++++++++++++++
 .../impl/GridTcpRouterNioListenerAdapter.java   |  2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  3 ++-
 .../ignite/internal/util/IgniteUtils.java       | 21 ++++++++++++++++++++
 6 files changed, 48 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[04/17] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-6' into ignite-1020

Posted by vk...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-6' into ignite-1020


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

Branch: refs/heads/ignite-884
Commit: 9a0e659a847ab3ec7c03443e2ada28aac51a26ea
Parents: 7fd1441 d1a2150
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 17 10:25:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 17 10:25:34 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |   2 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |  22 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   | 290 +++++++++++++++++++
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   6 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   | 148 ++++++++++
 .../configuration/CacheConfiguration.java       |   3 +-
 .../internal/interop/InteropBootstrap.java      |   3 +-
 .../internal/interop/InteropIgnition.java       |   5 +-
 .../processors/cache/GridCacheProcessor.java    |  23 +-
 .../util/spring/IgniteSpringHelper.java         |  10 +
 .../SpringApplicationContextResource.java       |   4 +-
 .../apache/ignite/resources/SpringResource.java |   6 +-
 modules/hibernate/pom.xml                       |  14 +
 .../hibernate/CacheHibernateBlobStore.java      |  87 +-----
 .../CacheHibernateBlobStoreFactory.java         | 235 +++++++++++++++
 .../hibernate/src/test/config/factory-cache.xml |  59 ++++
 .../src/test/config/factory-cache1.xml          |  61 ++++
 .../config/factory-incorrect-store-cache.xml    |  56 ++++
 .../CacheHibernateStoreFactorySelfTest.java     | 273 +++++++++++++++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 modules/spring/pom.xml                          |   7 +
 .../GridResourceSpringBeanInjector.java         |   2 +-
 .../util/spring/IgniteSpringHelperImpl.java     |  17 ++
 .../src/test/config/incorrect-store-cache.xml   |  57 ++++
 modules/spring/src/test/config/node.xml         |  43 +++
 modules/spring/src/test/config/node1.xml        |  45 +++
 .../test/config/pojo-incorrect-store-cache.xml  |  56 ++++
 modules/spring/src/test/config/store-cache.xml  |  59 ++++
 modules/spring/src/test/config/store-cache1.xml |  62 ++++
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java | 172 +++++++++++
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java | 193 ++++++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   5 +
 32 files changed, 1907 insertions(+), 120 deletions(-)
----------------------------------------------------------------------



[03/17] incubator-ignite git commit: #IGNITE-GG-10429 Fixed client marshaller.

Posted by vk...@apache.org.
#IGNITE-GG-10429 Fixed client marshaller.


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

Branch: refs/heads/ignite-884
Commit: 092bb91f1cf8b99b0f43903fd493d85cac89ee7e
Parents: 7785544
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 10:17:49 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 10:17:49 2015 +0300

----------------------------------------------------------------------
 .../internal/client/GridClientConfiguration.java | 19 +++++++++++++++++--
 .../optimized/GridClientOptimizedMarshaller.java |  5 +++--
 2 files changed, 20 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/092bb91f/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
index ee16f94..4a909a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
@@ -23,10 +23,12 @@ import org.apache.ignite.internal.client.marshaller.optimized.*;
 import org.apache.ignite.internal.client.ssl.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.plugin.*;
 import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import java.net.*;
+import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 
@@ -110,7 +112,7 @@ public class GridClientConfiguration {
     private ExecutorService executor;
 
     /** Marshaller. */
-    private GridClientMarshaller marshaller = new GridClientOptimizedMarshaller();
+    private GridClientMarshaller marshaller;
 
     /** Daemon flag. */
     private boolean daemon;
@@ -119,7 +121,20 @@ public class GridClientConfiguration {
      * Creates default configuration.
      */
     public GridClientConfiguration() {
-        // No-op.
+        List<PluginProvider> plugins = AccessController.doPrivileged(new PrivilegedAction<List<PluginProvider>>() {
+            @Override public List<PluginProvider> run() {
+                List<PluginProvider> providers = new ArrayList<>();
+
+                ServiceLoader<PluginProvider> ldr = ServiceLoader.load(PluginProvider.class);
+
+                for (PluginProvider provider : ldr)
+                    providers.add(provider);
+
+                return providers;
+            }
+        });
+
+        marshaller = new GridClientOptimizedMarshaller(plugins);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/092bb91f/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
index 28b129e..0d99d75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.client.marshaller.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.plugin.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.nio.*;
@@ -53,7 +54,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
      *
      * @param plugins Plugins.
      */
-    public GridClientOptimizedMarshaller(List<PluginProvider> plugins) {
+    public GridClientOptimizedMarshaller(@Nullable List<PluginProvider> plugins) {
         opMarsh = new OptimizedMarshaller();
 
         opMarsh.setContext(new ClientMarshallerContext(plugins));
@@ -121,7 +122,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
         /**
          * @param plugins Plugins.
          */
-        public ClientMarshallerContext(List<PluginProvider> plugins) {
+        public ClientMarshallerContext(@Nullable List<PluginProvider> plugins) {
             super(plugins);
         }
 


[07/17] incubator-ignite git commit: 1.1.3-SNAPSHOT

Posted by vk...@apache.org.
1.1.3-SNAPSHOT


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

Branch: refs/heads/ignite-884
Commit: dc05e88c8a0dd314ebfba6b5baf246e0d00be6a2
Parents: d1a2150
Author: Ignite Teamcity <ig...@apache.org>
Authored: Wed Jun 17 12:04:20 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Wed Jun 17 12:04:20 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                  | 2 +-
 modules/aop/pom.xml                               | 2 +-
 modules/aws/pom.xml                               | 2 +-
 modules/clients/pom.xml                           | 2 +-
 modules/cloud/pom.xml                             | 2 +-
 modules/codegen/pom.xml                           | 2 +-
 modules/core/pom.xml                              | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 modules/extdata/p2p/pom.xml                       | 2 +-
 modules/extdata/uri/pom.xml                       | 2 +-
 modules/gce/pom.xml                               | 2 +-
 modules/geospatial/pom.xml                        | 2 +-
 modules/hadoop/pom.xml                            | 2 +-
 modules/hibernate/pom.xml                         | 2 +-
 modules/indexing/pom.xml                          | 2 +-
 modules/jcl/pom.xml                               | 2 +-
 modules/jta/pom.xml                               | 2 +-
 modules/log4j/pom.xml                             | 2 +-
 modules/mesos/pom.xml                             | 2 +-
 modules/rest-http/pom.xml                         | 2 +-
 modules/scalar-2.10/pom.xml                       | 2 +-
 modules/scalar/pom.xml                            | 2 +-
 modules/schedule/pom.xml                          | 2 +-
 modules/schema-import/pom.xml                     | 2 +-
 modules/slf4j/pom.xml                             | 2 +-
 modules/spark-2.10/pom.xml                        | 2 +-
 modules/spark/pom.xml                             | 2 +-
 modules/spring/pom.xml                            | 2 +-
 modules/ssh/pom.xml                               | 2 +-
 modules/tools/pom.xml                             | 2 +-
 modules/urideploy/pom.xml                         | 2 +-
 modules/visor-console-2.10/pom.xml                | 2 +-
 modules/visor-console/pom.xml                     | 2 +-
 modules/visor-plugins/pom.xml                     | 2 +-
 modules/web/pom.xml                               | 2 +-
 modules/yardstick/pom.xml                         | 2 +-
 pom.xml                                           | 2 +-
 37 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 016b3ce..8422402 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index e04f183..1119301 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index f7fdfbc..3b1f3c6 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 7842c47..dfde946 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 579c6a0..b45b503 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 9d375cc..94154ea 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index a0ee32e..b81cc63 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 5171bd6..32d6d1f 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.1.2-SNAPSHOT
+ignite.version=1.1.3-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 4546112..0ee5e75 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index be0c73b..633df8b 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index e0a5d1d..2da3a6c 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index d9bd236..4f71637 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index dd7c9b7..ef26445 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index d7e098c..b4da498 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 18a3579..451e801 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 516df8a..0b0f2a6 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index ca5ef52..b4d6a5a 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 533f592..2c5e15e 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 173fe7c..d718a65 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 243ac06..18ec434 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 63bfd36..71c1ec9 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 3fb2470..635b874 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 049fe65..0c73aa0 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 8523d34..92b9da8 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 5284182..22f538f 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 46cc4e7..5e5d78e 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index e62ca24..244fa3c 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index d56cb2b..85f57bb 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index d65d173..dcd74d8 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 0bb221f..41e4f8a 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 70a23b0..7970f4b 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index bb7aefc..f5116bd 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index ef6abc0..885afe1 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index ef217ed..51680fa 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 6b69c63..97901ba 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 39c7f5a..c4cb122 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dc05e88c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 68d610e..84e1c79 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.3-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[16/17] incubator-ignite git commit: IGNITE-884 - Spring autowiring

Posted by vk...@apache.org.
IGNITE-884 - Spring autowiring


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

Branch: refs/heads/ignite-884
Commit: aaa7818e03b182744960ec3babfb215d306d5878
Parents: e011011
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 17 18:45:17 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 17 18:45:17 2015 -0700

----------------------------------------------------------------------
 .../cache/DynamicCacheChangeRequest.java           | 17 +++++++++++++++++
 .../processors/cache/GridCacheProcessor.java       | 16 +++++++++++-----
 .../optimized/OptimizedClassDescriptor.java        |  2 +-
 3 files changed, 29 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aaa7818e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index c08a179..cfe61c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -63,6 +63,9 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Template configuration flag. */
     private boolean template;
 
+    /** Start context. */
+    private transient CacheStartContext startCtx;
+
     /**
      * Constructor creates cache stop request.
      *
@@ -220,6 +223,20 @@ public class DynamicCacheChangeRequest implements Serializable {
         this.failIfExists = failIfExists;
     }
 
+    /**
+     * @return Start context.
+     */
+    public CacheStartContext startContext() {
+        return startCtx;
+    }
+
+    /**
+     * @param startCtx Start context.
+     */
+    public void startContext(CacheStartContext startCtx) {
+        this.startCtx = startCtx;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheChangeRequest.class, this, "cacheName", cacheName());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aaa7818e/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 a86738f..441f0f2 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
@@ -226,7 +226,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         if (cfg.getCacheStoreFactory() == null) {
-            Factory<CacheLoader> ldrFactory = cfg.isReadThrough() ? cfg.getCacheLoaderFactory() : null;
+            Factory<CacheLoader> ldrFactory = cfg.getCacheLoaderFactory();
             Factory<CacheWriter> writerFactory = cfg.isWriteThrough() ? cfg.getCacheWriterFactory() : null;
 
             if (ldrFactory != null || writerFactory != null)
@@ -1994,7 +1994,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             return new GridFinishedFuture<>(e);
         }
 
-        startCtxs.putIfAbsent(maskNull(cacheName), new CacheStartContext(ctx, cfg));
+        req.startContext(new CacheStartContext(ctx, cfg));
 
         return F.first(initiateCacheChanges(F.asList(req), failIfExists));
     }
@@ -2026,8 +2026,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             DynamicCacheStartFuture fut = new DynamicCacheStartFuture(req.cacheName(), req.deploymentId(), req);
 
             try {
+                String masked = maskNull(req.cacheName());
+
                 if (req.stop()) {
-                    DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
+                    DynamicCacheDescriptor desc = registeredCaches.get(masked);
 
                     if (desc == null)
                         // No-op.
@@ -2046,8 +2048,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 if (fut.isDone())
                     continue;
 
-                DynamicCacheStartFuture old = (DynamicCacheStartFuture)pendingFuts.putIfAbsent(
-                    maskNull(req.cacheName()), fut);
+                DynamicCacheStartFuture old = (DynamicCacheStartFuture)pendingFuts.putIfAbsent(masked, fut);
 
                 if (old != null) {
                     if (req.start()) {
@@ -2074,6 +2075,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         continue;
                     }
                 }
+                else {
+                    CacheStartContext oldStartCtx = startCtxs.put(masked, req.startContext());
+
+                    assert oldStartCtx == null;
+                }
 
                 if (fut.isDone())
                     continue;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aaa7818e/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index b66de27..05ef534 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -531,7 +531,7 @@ class OptimizedClassDescriptor {
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     void write(OptimizedObjectOutputStream out, Object obj) throws IOException {
-        out.writeByte(type);
+        out.write(type);
 
         switch (type) {
             case BYTE:


[02/17] incubator-ignite git commit: #IGNITE-GG-10429 Fixed client rest.

Posted by vk...@apache.org.
#IGNITE-GG-10429 Fixed client rest.


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

Branch: refs/heads/ignite-884
Commit: 77855440af71299a29aef0b36627044b705724b8
Parents: d1a2150
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 09:48:03 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 09:48:03 2015 +0300

----------------------------------------------------------------------
 .../GridClientOptimizedMarshaller.java          | 20 ++++++++++++++++++++
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  3 ++-
 2 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/77855440/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
index 7c2bd2f..28b129e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
@@ -22,9 +22,11 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.client.marshaller.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
 import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.plugin.*;
 
 import java.io.*;
 import java.nio.*;
+import java.util.*;
 
 /**
  * Wrapper, that adapts {@link org.apache.ignite.marshaller.optimized.OptimizedMarshaller} to
@@ -47,6 +49,17 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
     }
 
     /**
+     * Constructor.
+     *
+     * @param plugins Plugins.
+     */
+    public GridClientOptimizedMarshaller(List<PluginProvider> plugins) {
+        opMarsh = new OptimizedMarshaller();
+
+        opMarsh.setContext(new ClientMarshallerContext(plugins));
+    }
+
+    /**
      * Constructs optimized marshaller with specific parameters.
      *
      * @param requireSer Require serializable flag.
@@ -105,6 +118,13 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
             super(null);
         }
 
+        /**
+         * @param plugins Plugins.
+         */
+        public ClientMarshallerContext(List<PluginProvider> plugins) {
+            super(plugins);
+        }
+
         /** {@inheritDoc} */
         @Override protected boolean registerClassName(int id, String clsName) {
             throw new UnsupportedOperationException(clsName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/77855440/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 5645cf5..9b6f353 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -149,7 +149,8 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
 
         Map<Byte, GridClientMarshaller> marshMap = new HashMap<>();
 
-        marshMap.put(GridClientOptimizedMarshaller.ID, new GridClientOptimizedMarshaller());
+        marshMap.put(GridClientOptimizedMarshaller.ID,
+            new GridClientOptimizedMarshaller(new ArrayList<>(ctx.plugins().allProviders())));
         marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller());
 
         lsnr.marshallers(marshMap);