Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 6C7B12009C6 for ; Mon, 16 May 2016 10:11:50 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 6B40E160A19; Mon, 16 May 2016 08:11:50 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C20C41609B0 for ; Mon, 16 May 2016 10:11:48 +0200 (CEST) Received: (qmail 96523 invoked by uid 500); 16 May 2016 08:11:48 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 96513 invoked by uid 99); 16 May 2016 08:11:48 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 16 May 2016 08:11:48 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CA9CFE01BD; Mon, 16 May 2016 08:11:47 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: dmagda@apache.org To: commits@ignite.apache.org Date: Mon, 16 May 2016 08:11:47 -0000 Message-Id: <62bdd1ae0e764c4185e5dc1224b4970a@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [01/18] ignite git commit: IGNITE-2954 - Added tests for various read-through scenarios archived-at: Mon, 16 May 2016 08:11:50 -0000 Repository: ignite Updated Branches: refs/heads/ignite-3098 d43b71c52 -> 9e167073f IGNITE-2954 - Added tests for various read-through scenarios Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c0f64277 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c0f64277 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c0f64277 Branch: refs/heads/ignite-3098 Commit: c0f64277ff7199e7a096a61b4b18996b660e39ce Parents: bfba2a3 Author: Alexey Goncharuk Authored: Mon Apr 11 15:03:12 2016 -0700 Committer: Alexey Goncharuk Committed: Mon Apr 11 15:03:12 2016 -0700 ---------------------------------------------------------------------- .../processors/cache/GridCacheMapEntry.java | 3 +- ...ReadThroughEvictionOffheapTiredSelfTest.java | 30 -- .../IgniteCacheReadThroughEvictionSelfTest.java | 300 ++++++++----------- ...acheReadThroughEvictionsVariationsSuite.java | 58 ++++ .../ConfigVariationsTestSuiteBuilder.java | 18 +- .../configvariations/VariationsTestsConfig.java | 19 +- ...IgniteCacheConfigVariationsAbstractTest.java | 3 +- .../IgniteConfigVariationsAbstractTest.java | 6 + .../testsuites/IgniteCacheTestSuite5.java | 3 + 9 files changed, 220 insertions(+), 220 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java index 735e20a..dd62698 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java @@ -3748,7 +3748,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme synchronized (this) { checkObsolete(); - updateTtl(ttl); + if (hasValueUnlocked()) + updateTtl(ttl); /* TODO IGNITE-305. http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionOffheapTiredSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionOffheapTiredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionOffheapTiredSelfTest.java deleted file mode 100644 index a8ad4ee..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionOffheapTiredSelfTest.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache; - -import org.apache.ignite.cache.CacheMemoryMode; - -/** - * - */ -public class IgniteCacheReadThroughEvictionOffheapTiredSelfTest extends IgniteCacheReadThroughEvictionSelfTest { - /** {@inheritDoc} */ - @Override protected CacheMemoryMode cacheMemoryMode() { - return CacheMemoryMode.OFFHEAP_TIERED; - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionSelfTest.java index 1d8de74..78afdc8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionSelfTest.java @@ -20,153 +20,77 @@ package org.apache.ignite.internal.processors.cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMemoryMode; -import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; -import org.apache.ignite.cache.store.CacheStore; -import org.apache.ignite.cache.store.CacheStoreAdapter; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; -import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testframework.junits.IgniteCacheConfigVariationsAbstractTest; -import javax.cache.Cache; import javax.cache.configuration.Factory; import javax.cache.expiry.AccessedExpiryPolicy; import javax.cache.expiry.CreatedExpiryPolicy; import javax.cache.expiry.Duration; import javax.cache.expiry.ExpiryPolicy; -import javax.cache.integration.CacheLoaderException; -import javax.cache.integration.CacheWriterException; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; -import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; -import static org.apache.ignite.cache.CacheMode.PARTITIONED; - /** * */ -public class IgniteCacheReadThroughEvictionSelfTest extends GridCommonAbstractTest { +@SuppressWarnings("unchecked") +public class IgniteCacheReadThroughEvictionSelfTest extends IgniteCacheConfigVariationsAbstractTest { /** */ - private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + private static final int TIMEOUT = 400; /** */ - private static final int GRID_CNT = 5; - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - TcpDiscoverySpi disco = new TcpDiscoverySpi(); - - disco.setIpFinder(IP_FINDER); - - cfg.setDiscoverySpi(disco); - - if (F.eq(gridName, getTestGridName(GRID_CNT - 1))) - cfg.setClientMode(true); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - startGrids(GRID_CNT); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(false); - } + private static final int KEYS = 100; /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { - TestStore.storeMap.clear(); - } - - /** - * @return Cache configuration. - */ - private CacheConfiguration cacheConfiguration() { - CacheConfiguration cc = new CacheConfiguration(); - - cc.setAtomicityMode(atomicityMode()); - cc.setLoadPreviousValue(false); - cc.setCacheMode(cacheMode()); - cc.setMemoryMode(cacheMemoryMode()); - cc.setReadThrough(true); - cc.setWriteThrough(true); - cc.setBackups(1); - - cc.setCacheStoreFactory(new TestStoreFactory()); - - return cc; - } - - /** {@inheritDoc} */ - protected CacheAtomicityMode atomicityMode() { - return TRANSACTIONAL; - } - - /** {@inheritDoc} */ - protected CacheMode cacheMode() { - return PARTITIONED; - } - - /** {@inheritDoc} */ - protected CacheMemoryMode cacheMemoryMode() { - return CacheMemoryMode.ONHEAP_TIERED; + resetStore(); } /** * @throws Exception if failed. */ - @SuppressWarnings("unchecked") public void testReadThroughWithExpirePolicy() throws Exception { - Ignite ig = ignite(GRID_CNT - 1); + Ignite ig = testedGrid(); - CacheConfiguration cc = cacheConfiguration(); + CacheConfiguration cc = variationConfig("expire"); - IgniteCache cache = ig.getOrCreateCache(cc); + IgniteCache cache = ig.createCache(cc); try { - ExpiryPolicy exp = new CreatedExpiryPolicy(new Duration(TimeUnit.SECONDS, 1)); + ExpiryPolicy exp = new CreatedExpiryPolicy(new Duration(TimeUnit.MILLISECONDS, TIMEOUT)); - for (int i = 0; i < 1000; i++) - cache.withExpiryPolicy(exp).put(i, i); + for (int i = 0; i < KEYS; i++) + cache.withExpiryPolicy(exp).put(key(i), value(i)); - U.sleep(1_000); + U.sleep(TIMEOUT); - waitEmpty(); + waitEmpty(cc.getName()); - exp = new AccessedExpiryPolicy(new Duration(TimeUnit.SECONDS, 1)); + exp = new AccessedExpiryPolicy(new Duration(TimeUnit.MILLISECONDS, TIMEOUT)); - for (int i = 0; i < 1000; i++) { - assertEquals(i, ig.cache(null).get(i)); + for (int i = 0; i < KEYS; i++) { + assertEquals(value(i), cache.get(key(i))); - ig.cache(null).withExpiryPolicy(exp).get(i); + cache.withExpiryPolicy(exp).get(key(i)); } - U.sleep(1_000); + U.sleep(TIMEOUT); - waitEmpty(); + waitEmpty(cc.getName()); - for (int i = 0; i < 1000; i++) - assertEquals(i, ig.cache(null).get(i)); + for (int i = 0; i < KEYS; i++) + assertEquals(value(i), cache.get(key(i))); } finally { - ig.destroyCache(null); + destroyCacheSafe(ig, cc.getName()); } } @@ -174,54 +98,52 @@ public class IgniteCacheReadThroughEvictionSelfTest extends GridCommonAbstractTe * @throws Exception if failed. */ public void testReadThroughExpirePolicyConfigured() throws Exception { - Ignite ig = ignite(GRID_CNT - 1); + Ignite ig = testedGrid(); - CacheConfiguration cc = cacheConfiguration(); + CacheConfiguration cc = variationConfig("expireConfig"); cc.setExpiryPolicyFactory(new ExpirePolicyFactory()); - IgniteCache cache = ig.getOrCreateCache(cc); + IgniteCache cache = ig.createCache(cc); try { - int keyCnt = 1000; + for (int i = 0; i < KEYS; i++) + cache.put(key(i), value(i)); - for (int i = 0; i < keyCnt; i++) - cache.put(i, i); + U.sleep(TIMEOUT); - U.sleep(1_000); + waitEmpty(cc.getName()); - waitEmpty(); - - for (int i = 0; i < keyCnt; i++) { - assertEquals(i, ig.cache(null).get(i)); + for (int i = 0; i < KEYS; i++) { + assertEquals(value(i), cache.get(key(i))); // Access expiry. - ig.cache(null).get(i); + cache.get(key(i)); } - U.sleep(1_000); + U.sleep(TIMEOUT); - waitEmpty(); + waitEmpty(cc.getName()); - for (int i = 0; i < keyCnt; i++) - assertEquals(i, ig.cache(null).get(i)); + for (int i = 0; i < KEYS; i++) + assertEquals(value(i), cache.get(key(i))); - for (int i = 0; i < keyCnt; i++) { - assertEquals(i, ig.cache(null).get(i)); + for (int i = 0; i < KEYS; i++) { + assertEquals(value(i), cache.get(key(i))); // Update expiry. - ig.cache(null).put(i, i); + cache.put(key(i), value(i)); } - U.sleep(1_000); + U.sleep(TIMEOUT); - waitEmpty(); + waitEmpty(cc.getName()); - for (int i = 0; i < keyCnt; i++) - assertEquals(i, ig.cache(null).get(i)); + for (int i = 0; i < KEYS; i++) + assertEquals(value(i), cache.get(key(i))); } finally { - ig.destroyCache(null); + destroyCacheSafe(ig, cc.getName()); } } @@ -229,33 +151,37 @@ public class IgniteCacheReadThroughEvictionSelfTest extends GridCommonAbstractTe * @throws Exception if failed. */ public void testReadThroughEvictionPolicy() throws Exception { - Ignite ig = ignite(GRID_CNT - 1); + Ignite ig = testedGrid(); - CacheConfiguration cc = cacheConfiguration(); + CacheConfiguration cc = variationConfig("eviction"); cc.setEvictionPolicy(new FifoEvictionPolicy(1)); - final IgniteCache cache = ig.getOrCreateCache(cc); + if (cc.getMemoryMode() == CacheMemoryMode.OFFHEAP_TIERED) + cc.setOffHeapMaxMemory(2 * 1024); + + final IgniteCache cache = ig.createCache(cc); try { - for (int i = 0; i < 1000; i++) - cache.put(i, i); + for (int i = 0; i < KEYS; i++) + cache.put(key(i), value(i)); assertTrue(GridTestUtils.waitForCondition(new PA() { @Override public boolean apply() { - int size = cache.size(); + int size = cache.size(CachePeekMode.ONHEAP); + int offheapSize = cache.size(CachePeekMode.OFFHEAP); - System.out.println("Cache size: " + size); + System.out.println("Cache [onHeap=" + size + ", offHeap=" + offheapSize + ']'); - return size <= GRID_CNT; + return size <= testsCfg.gridCount() && offheapSize < KEYS; } }, getTestTimeout())); - for (int i = 0; i < 1000; i++) - assertEquals(i, cache.get(i)); + for (int i = 0; i < KEYS; i++) + assertEquals(value(i), cache.get(key(i))); } finally { - ig.destroyCache(null); + destroyCacheSafe(ig, cc.getName()); } } @@ -263,37 +189,56 @@ public class IgniteCacheReadThroughEvictionSelfTest extends GridCommonAbstractTe * @throws Exception if failed. */ public void testReadThroughSkipStore() throws Exception { - Ignite ig = ignite(GRID_CNT - 1); + Ignite ig = testedGrid(); - CacheConfiguration cc = cacheConfiguration(); + CacheConfiguration cc = variationConfig("skipStore"); - final IgniteCache cache = ig.getOrCreateCache(cc); + final IgniteCache cache = ig.createCache(cc); try { - for (int i = 0; i < 1000; i++) { - cache.put(i, i); + for (int i = 0; i < KEYS; i++) { + cache.put(key(i), value(i)); - cache.withSkipStore().remove(i); + cache.withSkipStore().remove(key(i)); } - waitEmpty(); + waitEmpty(cc.getName()); - for (int i = 0; i < 1000; i++) - assertEquals(i, cache.get(i)); + for (int i = 0; i < KEYS; i++) + assertEquals(value(i), cache.get(key(i))); } finally { - ig.destroyCache(null); + destroyCacheSafe(ig, cc.getName()); } } /** + * @return Variation test configuration. + */ + private CacheConfiguration variationConfig(String suffix) { + CacheConfiguration ccfg = testsCfg.configurationFactory().cacheConfiguration(getTestGridName(testedNodeIdx)); + + ccfg.setName(cacheName() + "_" + suffix); + + return ccfg; + } + + /** * @throws Exception if failed. */ - private void waitEmpty() throws Exception { + private void waitEmpty(final String name) throws Exception { boolean success = GridTestUtils.waitForCondition(new PA() { @Override public boolean apply() { - for (int g = 0; g < GRID_CNT - 1; g++) { - if (!grid(g).context().cache().internalCache().isEmpty()) + for (Ignite g : G.allGrids()) { + GridCacheAdapter cache = ((IgniteEx)g).context().cache().internalCache(name); + + if (cache == null) + return true; + + if (!cache.isEmpty()) + return false; + + if (cache.context().offheap().entriesCount(null) > 0) return false; } @@ -305,55 +250,48 @@ public class IgniteCacheReadThroughEvictionSelfTest extends GridCommonAbstractTe } /** - * + * @param ig Ignite. + * @param cacheName Cache name to destroy. + * @throws IgniteCheckedException If failed. */ - private static class TestStoreFactory implements Factory { - @Override public CacheStore create() { - return new TestStore(); - } + private void destroyCacheSafe(Ignite ig, final String cacheName) throws IgniteCheckedException { + ig.destroyCache(cacheName); + + GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + for (Ignite g : G.allGrids()) { + IgniteInternalCache cache = ((IgniteEx)g).context().cache().cache(cacheName); + + if (cache != null) + return false; + } + + return true; + } + }, getTestTimeout()); } + /** + * + */ private static class ExpirePolicyFactory implements Factory { /** {@inheritDoc} */ @Override public ExpiryPolicy create() { return new ExpiryPolicy() { @Override public Duration getExpiryForCreation() { - return new Duration(TimeUnit.SECONDS, 1); + return new Duration(TimeUnit.MILLISECONDS, TIMEOUT); } /** {@inheritDoc} */ @Override public Duration getExpiryForAccess() { - return new Duration(TimeUnit.SECONDS, 1); + return new Duration(TimeUnit.MILLISECONDS, TIMEOUT); } /** {@inheritDoc} */ @Override public Duration getExpiryForUpdate() { - return new Duration(TimeUnit.SECONDS, 1); + return new Duration(TimeUnit.MILLISECONDS, TIMEOUT); } }; } } - - /** - * - */ - private static class TestStore extends CacheStoreAdapter { - /** */ - private static Map storeMap = new ConcurrentHashMap<>(); - - /** {@inheritDoc} */ - @Override public Object load(Object key) throws CacheLoaderException { - return storeMap.get(key); - } - - /** {@inheritDoc} */ - @Override public void write(Cache.Entry entry) throws CacheWriterException { - storeMap.put(entry.getKey(), entry.getValue()); - } - - /** {@inheritDoc} */ - @Override public void delete(Object key) throws CacheWriterException { - storeMap.remove(key); - } - } } http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionsVariationsSuite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionsVariationsSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionsVariationsSuite.java new file mode 100644 index 0000000..400b2e9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughEvictionsVariationsSuite.java @@ -0,0 +1,58 @@ +/* + * 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 junit.framework.TestSuite; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.configvariations.ConfigVariationsTestSuiteBuilder; + +/** + * + */ +public class IgniteCacheReadThroughEvictionsVariationsSuite extends TestSuite { + /** + * @return Cache API test suite. + * @throws Exception If failed. + */ + public static TestSuite suite() throws Exception { + return new ConfigVariationsTestSuiteBuilder( + "Cache Read Through Variations Test", + IgniteCacheReadThroughEvictionSelfTest.class) + .withBasicCacheParams() + .withIgniteConfigFilters(new IgnitePredicate() { + /** {@inheritDoc} */ + @SuppressWarnings("RedundantIfStatement") + @Override public boolean apply(IgniteConfiguration cfg) { + if (cfg.getMarshaller() != null && !(cfg.getMarshaller() instanceof BinaryMarshaller)) + return false; + + if (cfg.isPeerClassLoadingEnabled()) + return false; + + return true; + } + }) + .skipWaitPartitionMapExchange() + .gridsCount(4).backups(1) + .testedNodesCount(2).withClients() + .build(); + } + +} http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java index 71d7987..5c88f4e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java @@ -72,6 +72,9 @@ public class ConfigVariationsTestSuiteBuilder { /** */ private IgnitePredicate[] cacheCfgFilters; + /** */ + private boolean skipWaitPartMapExchange; + /** * @param name Name. * @param cls Test class. @@ -189,13 +192,13 @@ public class ConfigVariationsTestSuiteBuilder { + ", cacheCfg=" + factory.getCacheConfigurationDescription() + "]"; VariationsTestsConfig testCfg = new VariationsTestsConfig(factory, clsNameSuffix, stopNodes, cacheStartMode, - gridsCnt); + gridsCnt, !skipWaitPartMapExchange); TestSuite addedSuite; if (testedNodeCnt > 1) addedSuite = createMultiNodeTestSuite((Class)cls, - testCfg, testedNodeCnt, withClients); + testCfg, testedNodeCnt, withClients, skipWaitPartMapExchange); else addedSuite = new IgniteConfigVariationsTestSuite(cls, testCfg); @@ -208,7 +211,7 @@ public class ConfigVariationsTestSuiteBuilder { * @param testedNodeCnt Count of tested nodes. */ private static TestSuite createMultiNodeTestSuite(Class cls, - VariationsTestsConfig cfg, int testedNodeCnt, boolean withClients) { + VariationsTestsConfig cfg, int testedNodeCnt, boolean withClients, boolean skipWaitParMapExchange) { TestSuite suite = new TestSuite(); if (cfg.gridCount() < testedNodeCnt) @@ -221,7 +224,8 @@ public class ConfigVariationsTestSuiteBuilder { boolean stopCache = i + 1 == testedNodeCnt; VariationsTestsConfig cfg0 = new VariationsTestsConfig(cfg.configurationFactory(), cfg.description(), - stopNodes, startCache, stopCache, cfg.cacheStartMode(), cfg.gridCount(), i, withClients); + stopNodes, startCache, stopCache, cfg.cacheStartMode(), cfg.gridCount(), i, withClients, + !skipWaitParMapExchange); suite.addTest(new IgniteConfigVariationsTestSuite(cls, cfg0)); } @@ -338,6 +342,12 @@ public class ConfigVariationsTestSuiteBuilder { return this; } + public ConfigVariationsTestSuiteBuilder skipWaitPartitionMapExchange() { + skipWaitPartMapExchange = true; + + return this; + } + /** * @param filters Ignite configuration filters. * @return {@code this} for chaining. http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java index 7bcfc7f..0395aa9 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java @@ -48,6 +48,9 @@ public class VariationsTestsConfig { private boolean stopCache; /** */ + private boolean awaitPartMapExchange = true; + + /** */ private boolean withClients; /** @@ -61,9 +64,10 @@ public class VariationsTestsConfig { String desc, boolean stopNodes, CacheStartMode cacheStartMode, - int gridCnt + int gridCnt, + boolean awaitPartMapExchange ) { - this(factory, desc, stopNodes, true, true, cacheStartMode, gridCnt, 0, false); + this(factory, desc, stopNodes, true, true, cacheStartMode, gridCnt, 0, false, awaitPartMapExchange); } /** @@ -81,7 +85,8 @@ public class VariationsTestsConfig { CacheStartMode cacheStartMode, int gridCnt, int testedNodeIdx, - boolean withClients + boolean withClients, + boolean awaitPartMapExchange ) { A.ensure(gridCnt >= 1, "Grids count cannot be less then 1."); @@ -94,6 +99,7 @@ public class VariationsTestsConfig { this.startCache = startCache; this.stopCache = stopCache; this.withClients = withClients; + this.awaitPartMapExchange = awaitPartMapExchange; } /** @@ -158,4 +164,11 @@ public class VariationsTestsConfig { public boolean withClients() { return withClients; } + + /** + * @return Partition map exchange wait flag. + */ + public boolean awaitPartitionMapExchange() { + return awaitPartMapExchange; + } } http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java index 28c6f55..acf4a05 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java @@ -198,7 +198,8 @@ public abstract class IgniteCacheConfigVariationsAbstractTest extends IgniteConf @Override protected void beforeTest() throws Exception { super.beforeTest(); - awaitPartitionMapExchange(); + if (testsCfg.awaitPartitionMapExchange()) + awaitPartitionMapExchange(); assert jcache().unwrap(Ignite.class).transactions().tx() == null; http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java index b22f289..4b94e34 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java @@ -234,6 +234,9 @@ public abstract class IgniteConfigVariationsAbstractTest extends GridCommonAbstr * @return Key. */ public static Object key(int keyId, DataMode mode) { + if (mode == null) + mode = DataMode.SERIALIZABLE; + switch (mode) { case SERIALIZABLE: return new SerializableObject(keyId); @@ -263,6 +266,9 @@ public abstract class IgniteConfigVariationsAbstractTest extends GridCommonAbstr * @return Value. */ public static Object value(int idx, DataMode mode) { + if (mode == null) + mode = DataMode.SERIALIZABLE; + switch (mode) { case SERIALIZABLE: return new SerializableObject(idx); http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f64277/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java index bb05a70..a263c0d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.CacheSerializableTransactions import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest; import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest; import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughEvictionsVariationsSuite; import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentFairAffinityTest; import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest; @@ -52,6 +53,8 @@ public class IgniteCacheTestSuite5 extends TestSuite { suite.addTestSuite(CacheLateAffinityAssignmentNodeJoinValidationTest.class); suite.addTestSuite(EntryVersionConsistencyReadThroughTest.class); + suite.addTest(IgniteCacheReadThroughEvictionsVariationsSuite.suite()); + return suite; } }