You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/17 16:24:03 UTC

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

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-6 5218210cf -> af829d0ee


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-sprint-6
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(),


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

Posted by sb...@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-sprint-6
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);


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

Posted by sb...@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-sprint-6
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(-)
----------------------------------------------------------------------



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

Posted by sb...@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-sprint-6
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(-)
----------------------------------------------------------------------



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

Posted by sb...@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-sprint-6
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);
 


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

Posted by sb...@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-sprint-6
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(-)
----------------------------------------------------------------------