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 54A19200C16 for ; Thu, 9 Feb 2017 18:43:05 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 530A5160B4B; Thu, 9 Feb 2017 17:43:05 +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 0EC93160B72 for ; Thu, 9 Feb 2017 18:43:02 +0100 (CET) Received: (qmail 55553 invoked by uid 500); 9 Feb 2017 17:43:02 -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 55233 invoked by uid 99); 9 Feb 2017 17:43:02 -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; Thu, 09 Feb 2017 17:43:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C1EE9DFB0E; Thu, 9 Feb 2017 17:43:01 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agura@apache.org To: commits@ignite.apache.org Date: Thu, 09 Feb 2017 17:43:08 -0000 Message-Id: <3f8f0833dfe34476af5c4268c4b7efbb@git.apache.org> In-Reply-To: <6535866b54cc435ebf126dd05dd2ff8b@git.apache.org> References: <6535866b54cc435ebf126dd05dd2ff8b@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [08/24] ignite git commit: ignite-4465 Prevent cache entry eviction while it is loaded from store (otherwise loaded value can be not stored in cache). archived-at: Thu, 09 Feb 2017 17:43:05 -0000 ignite-4465 Prevent cache entry eviction while it is loaded from store (otherwise loaded value can be not stored in cache). Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/22b7e76c Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/22b7e76c Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/22b7e76c Branch: refs/heads/master Commit: 22b7e76c00a77a06388bcef869f29d1a572a306a Parents: 7117647 Author: sboikov Authored: Tue Jan 17 12:33:32 2017 +0300 Committer: sboikov Committed: Tue Jan 17 12:33:32 2017 +0300 ---------------------------------------------------------------------- .../processors/cache/EntryGetResult.java | 65 +++++++ .../processors/cache/GridCacheAdapter.java | 104 ++++++----- .../processors/cache/GridCacheEntryEx.java | 30 ++- .../processors/cache/GridCacheMapEntry.java | 100 ++++++++-- .../dht/GridPartitionedGetFuture.java | 7 +- .../dht/GridPartitionedSingleGetFuture.java | 7 +- .../dht/atomic/GridDhtAtomicCache.java | 7 +- .../dht/colocated/GridDhtColocatedCache.java | 7 +- .../distributed/near/GridNearGetFuture.java | 13 +- .../local/atomic/GridLocalAtomicCache.java | 7 +- .../cache/transactions/IgniteTxHandler.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 29 +-- .../cache/CacheConcurrentReadThroughTest.java | 184 +++++++++++++++++++ .../cache/CrossCacheTxRandomOperationsTest.java | 28 ++- .../processors/cache/GridCacheTestEntryEx.java | 21 ++- .../testsuites/IgniteCacheTestSuite2.java | 2 + 16 files changed, 512 insertions(+), 101 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/22b7e76c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java new file mode 100644 index 0000000..a34ddae --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/EntryGetResult.java @@ -0,0 +1,65 @@ +/* + * 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.internal.processors.cache.version.GridCacheVersion; + +/** + * + */ +public class EntryGetResult { + /** */ + private final CacheObject val; + + /** */ + private final GridCacheVersion ver; + + /** */ + private final boolean reserved; + + /** + * @param val Value. + * @param ver Version. + */ + EntryGetResult(CacheObject val, GridCacheVersion ver, boolean reserved) { + this.val = val; + this.ver = ver; + this.reserved = reserved; + } + + /** + * @return Value. + */ + public CacheObject value() { + return val; + } + + /** + * @return Version. + */ + public GridCacheVersion version() { + return ver; + } + + /** + * @return Reserved flag, + */ + public boolean reserved() { + return reserved; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/22b7e76c/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 965c6d1..fd9f396 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 @@ -1860,7 +1860,7 @@ public abstract class GridCacheAdapter implements IgniteInternalCache IgniteInternalFuture> getAllAsync0( + protected final IgniteInternalFuture> getAllAsync0( @Nullable final Collection keys, final boolean readThrough, boolean checkTx, @@ -1906,7 +1906,7 @@ public abstract class GridCacheAdapter implements IgniteInternalCache misses = null; + Map misses = null; for (KeyCacheObject key : keys) { while (true) { @@ -1920,40 +1920,58 @@ public abstract class GridCacheAdapter implements IgniteInternalCache res = entry.innerGetVersioned( - null, - null, - ctx.isSwapOrOffheapEnabled(), - /*unmarshal*/true, - /*update-metrics*/!skipVals, - /*event*/!skipVals, - subjId, - null, - taskName, - expiry, - !deserializeBinary); - - if (res == null) { - if (storeEnabled) { - GridCacheVersion ver = entry.version(); + EntryGetResult res; + boolean evt = !skipVals; + boolean updateMetrics = !skipVals; + + if (storeEnabled) { + res = entry.innerGetAndReserveForLoad(ctx.isSwapOrOffheapEnabled(), + updateMetrics, + evt, + subjId, + taskName, + expiry, + !deserializeBinary); + + assert res != null; + + if (res.value() == null) { if (misses == null) misses = new HashMap<>(); - misses.put(key, ver); + misses.put(key, res); + + res = null; } - else - ctx.evicts().touch(entry, topVer); } else { + res = entry.innerGetVersioned( + null, + null, + ctx.isSwapOrOffheapEnabled(), + /*unmarshal*/true, + updateMetrics, + evt, + subjId, + null, + taskName, + expiry, + !deserializeBinary); + + if (res == null) + ctx.evicts().touch(entry, topVer); + } + + if (res != null) { ctx.addResult(map, key, - res.get1(), + res.value(), skipVals, keepCacheObjects, deserializeBinary, true, - needVer ? res.get2() : null); + needVer ? res.version() : null); if (tx == null || (!tx.implicit() && tx.isolation() == READ_COMMITTED)) ctx.evicts().touch(entry, topVer); @@ -1973,7 +1991,7 @@ public abstract class GridCacheAdapter implements IgniteInternalCache loadKeys = misses; + final Map loadKeys = misses; final IgniteTxLocalAdapter tx0 = tx; @@ -1984,15 +2002,10 @@ public abstract class GridCacheAdapter implements IgniteInternalCache call() throws Exception { ctx.store().loadAll(null/*tx*/, loadKeys.keySet(), new CI2() { @Override public void apply(KeyCacheObject key, Object val) { - GridCacheVersion ver = loadKeys.get(key); - - if (ver == null) { - if (log.isDebugEnabled()) - log.debug("Value from storage was never asked for [key=" + key + - ", val=" + val + ']'); + EntryGetResult res = loadKeys.get(key); + if (res == null || val == null) return; - } loaded.add(key); @@ -2002,14 +2015,16 @@ public abstract class GridCacheAdapter implements IgniteInternalCache> futs = new ArrayList<>(); + + for (int i = 0; i < SYS_THREADS * 3; i++) { + compute.run(new IgniteRunnable() { + @IgniteInstanceResource + private transient Ignite ignite; + + @Override public void run() { + assertFalse(ignite.configuration().isClientMode()); + + Object v = ignite.cache(cacheName).get(key); + + if (v == null) + throw new IgniteException("Failed to get value"); + } + }); + + futs.add(compute.future()); + } + + for (IgniteFuture fut : futs) + fut.get(); + + int loadCnt = TestCacheStore.loadCnt.get(); + + long misses = ignite(1).cache(cacheName).metrics().getCacheMisses(); + + log.info("Iteration [iter=" + iter + ", loadCnt=" + loadCnt + ", misses=" + misses + ']'); + + assertTrue("Unexpected loadCnt: " + loadCnt, loadCnt > 0 && loadCnt <= SYS_THREADS); + assertTrue("Unexpected misses: " + misses, misses > 0 && misses <= SYS_THREADS); + + client.destroyCache(cacheName); + } + } + + /** + * + */ + private static class TestStoreFactory implements Factory { + /** {@inheritDoc} */ + @Override public TestCacheStore create() { + return new TestCacheStore(); + } + } + + /** + * + */ + private static class TestCacheStore extends CacheStoreAdapter { + /** */ + private static final AtomicInteger loadCnt = new AtomicInteger(); + + /** {@inheritDoc} */ + @Override public Integer load(Integer key) throws CacheLoaderException { + loadCnt.incrementAndGet(); + + try { + Thread.sleep(1000); + } + catch (InterruptedException e) { + e.printStackTrace(); + } + + return key; + } + + /** {@inheritDoc} */ + @Override public void write(Cache.Entry entry) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void delete(Object key) { + // No-op. + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/22b7e76c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java index 67ec371..e7df3c0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java @@ -86,6 +86,11 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 6 * 60 * 1000; + } + + /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); @@ -170,9 +175,17 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest { } /** + * @param cacheMode Cache mode. + * @param writeSync Write synchronization mode. + * @param fairAff Fair affinity flag. + * @param ignite Node to use. + * @param name Cache name. */ - protected void createCache(CacheMode cacheMode, CacheWriteSynchronizationMode writeSync, boolean fairAff, - Ignite ignite, String name) { + protected void createCache(CacheMode cacheMode, + CacheWriteSynchronizationMode writeSync, + boolean fairAff, + Ignite ignite, + String name) { ignite.createCache(cacheConfiguration(name, cacheMode, writeSync, fairAff)); } @@ -269,9 +282,18 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest { boolean checkData = fullSync && !optimistic; + long stopTime = System.currentTimeMillis() + 10_000; + for (int i = 0; i < 10_000; i++) { - if (i % 100 == 0) + if (i % 100 == 0) { + if (System.currentTimeMillis() > stopTime) { + log.info("Stop on timeout, iteration: " + i); + + break; + } + log.info("Iteration: " + i); + } boolean rollback = i % 10 == 0; http://git-wip-us.apache.org/repos/asf/ignite/blob/22b7e76c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java index 48621af..b03e9c8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java @@ -418,7 +418,26 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr } /** @inheritDoc */ - @Nullable @Override public T2 innerGetVersioned( + @Override public void clearReserveForLoad(GridCacheVersion ver) { + assert false; + } + + /** @inheritDoc */ + @Override public EntryGetResult innerGetAndReserveForLoad( + boolean readSwap, + boolean updateMetrics, + boolean evt, + UUID subjId, + String taskName, + @Nullable IgniteCacheExpiryPolicy expiryPlc, + boolean keepBinary) throws IgniteCheckedException, GridCacheEntryRemovedException { + assert false; + + return null; + } + + /** @inheritDoc */ + @Nullable @Override public EntryGetResult innerGetVersioned( @Nullable GridCacheVersion ver, IgniteInternalTx tx, boolean readSwap, http://git-wip-us.apache.org/repos/asf/ignite/blob/22b7e76c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java index f632f67..8792ea1 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java @@ -22,6 +22,7 @@ import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionBackupFilterSel import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionExcludeNeighborsSelfTest; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest; +import org.apache.ignite.internal.processors.cache.CacheConcurrentReadThroughTest; import org.apache.ignite.internal.processors.cache.CacheConfigurationLeakTest; import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest; import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest; @@ -268,6 +269,7 @@ public class IgniteCacheTestSuite2 extends TestSuite { suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class)); suite.addTest(new TestSuite(OffheapCacheOnClientsTest.class)); + suite.addTest(new TestSuite(CacheConcurrentReadThroughTest.class)); return suite; }