From commits-return-116267-archive-asf-public=cust-asf.ponee.io@ignite.apache.org Thu Jan 11 13:11:00 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id 4B85C180787 for ; Thu, 11 Jan 2018 13:11:00 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 3B753160C1F; Thu, 11 Jan 2018 12:11:00 +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 0FCF8160C4A for ; Thu, 11 Jan 2018 13:10:57 +0100 (CET) Received: (qmail 60436 invoked by uid 500); 11 Jan 2018 12:10:57 -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 60310 invoked by uid 99); 11 Jan 2018 12:10:56 -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, 11 Jan 2018 12:10:56 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 0C465F32C7; Thu, 11 Jan 2018 12:10:52 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Thu, 11 Jan 2018 12:10:57 -0000 Message-Id: In-Reply-To: <8ab8fcf3ca67494297efece28c1fcadb@git.apache.org> References: <8ab8fcf3ca67494297efece28c1fcadb@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [07/17] ignite git commit: IGNITE-7173: Implemented SQL on-heap row cache. This closes #3291. IGNITE-7173: Implemented SQL on-heap row cache. This closes #3291. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/693a9593 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/693a9593 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/693a9593 Branch: refs/heads/ignite-zk Commit: 693a95933def14cd9ce6f48fae11dfe1cd68f026 Parents: 8e796dc Author: tledkov-gridgain Authored: Wed Jan 10 16:14:12 2018 +0300 Committer: devozerov Committed: Wed Jan 10 16:14:12 2018 +0300 ---------------------------------------------------------------------- .../configuration/CacheConfiguration.java | 29 ++ .../processors/cache/GridCacheProcessor.java | 2 +- .../cache/IgniteCacheOffheapManager.java | 8 + .../cache/IgniteCacheOffheapManagerImpl.java | 6 + .../distributed/dht/GridDhtLocalPartition.java | 11 + .../persistence/GridCacheOffheapManager.java | 14 + .../processors/cache/persistence/RowStore.java | 19 + .../persistence/pagemem/PageMemoryImpl.java | 57 ++- .../query/CacheQueryObjectValueContext.java | 4 +- .../processors/query/GridQueryIndexing.java | 12 +- .../processors/query/GridQueryProcessor.java | 14 +- .../query/GridQueryRowCacheCleaner.java | 30 ++ ...IgniteClientCacheInitializationFailTest.java | 8 +- .../processors/query/h2/H2RowCache.java | 153 ++++++++ .../processors/query/h2/H2RowCacheRegistry.java | 126 +++++++ .../processors/query/h2/IgniteH2Indexing.java | 21 +- .../processors/query/h2/database/H2Tree.java | 37 +- .../query/h2/database/H2TreeIndex.java | 8 +- .../query/h2/database/io/H2ExtrasInnerIO.java | 2 +- .../query/h2/database/io/H2ExtrasLeafIO.java | 2 +- .../query/h2/database/io/H2InnerIO.java | 2 +- .../query/h2/database/io/H2LeafIO.java | 2 +- .../cache/index/H2RowCachePageEvictionTest.java | 238 +++++++++++++ .../cache/index/H2RowCacheSelfTest.java | 352 +++++++++++++++++++ .../index/OptimizedMarshallerIndexNameTest.java | 9 +- .../IgniteCacheQuerySelfTestSuite.java | 7 +- 26 files changed, 1145 insertions(+), 28 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index 7d2730c..3a40824 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -210,6 +210,9 @@ public class CacheConfiguration extends MutableConfiguration { /** */ private boolean onheapCache; + /** Use on-heap cache for rows for SQL queries. */ + private boolean sqlOnheapCache; + /** Eviction filter. */ private EvictionFilter evictFilter; @@ -449,6 +452,7 @@ public class CacheConfiguration extends MutableConfiguration { writeSync = cc.getWriteSynchronizationMode(); storeConcurrentLoadAllThreshold = cc.getStoreConcurrentLoadAllThreshold(); maxQryIterCnt = cc.getMaxQueryIteratorsCount(); + sqlOnheapCache = cc.isSqlOnheapCacheEnabled(); } /** @@ -629,6 +633,31 @@ public class CacheConfiguration extends MutableConfiguration { } /** + * Gets whether SQL on-heap cache is enabled. When enabled, Ignite will cache SQL rows as they are accessed by + * query engine. Rows are invalidated and evicted from cache when relevant cache entry is either changed or + * evicted. + * + * @return Whether SQL onheap cache is enabled. + */ + public boolean isSqlOnheapCacheEnabled() { + return sqlOnheapCache; + } + + /** + * Sets whether SQL on-heap cache is enabled. When enabled, Ignite will cache SQL rows as they are accessed by + * query engine. Rows are invalidated and evicted from cache when relevant cache entry is either changed or + * evicted. + * + * @param sqlOnheapCache Whether SQL onheap cache is enabled. + * @return {@code this} for chaining. + */ + public CacheConfiguration setSqlOnheapCacheEnabled(boolean sqlOnheapCache) { + this.sqlOnheapCache = sqlOnheapCache; + + return this; + } + + /** * @return Near enabled flag. */ public NearCacheConfiguration getNearConfiguration() { http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 30033d3..4f849da 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -1065,7 +1065,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { boolean rmvIdx = !cache.context().group().persistenceEnabled(); - ctx.query().onCacheStop0(cctx.name(), rmvIdx); + ctx.query().onCacheStop0(cctx, rmvIdx); ctx.query().onCacheStart0(cctx, desc.schema()); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java index 84c69a9..546672c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.persistence.RowStore; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.IgniteTree; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -521,5 +522,12 @@ public interface IgniteCacheOffheapManager { * @param cntr Counter. */ void updateInitialCounter(long cntr); + + /** + * Inject rows cache cleaner. + * + * @param rowCacheCleaner Rows cache cleaner. + */ + public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 8ad6d4b..8793729 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.tree.PendingRow; import org.apache.ignite.internal.processors.cache.tree.SearchRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.GridEmptyCloseableIterator; @@ -1556,6 +1557,11 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager } /** {@inheritDoc} */ + @Override public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner) { + rowStore().setRowCacheCleaner(rowCacheCleaner); + } + + /** {@inheritDoc} */ @Override public void init(long size, long updCntr, @Nullable Map cacheSizes) { initCntr = updCntr; storageSize.set(size); http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java index c813a57..62e2b12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridIterator; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -197,6 +198,16 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements try { store = grp.offheap().createCacheDataStore(id); + + // Inject row cache cleaner on store creation + // Used in case the cache with enabled SqlOnheapCache is single cache at the cache group + if (ctx.kernalContext().query().moduleEnabled()) { + GridQueryRowCacheCleaner cleaner = ctx.kernalContext().query().getIndexing() + .rowCacheCleaner(grp.groupId()); + + if (store != null && cleaner != null) + store.setRowCacheCleaner(cleaner); + } } catch (IgniteCheckedException e) { // TODO ignite-db http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java index e818b00..5ccbfb9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java @@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore; import org.apache.ignite.internal.processors.cache.tree.CacheDataTree; import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -1226,6 +1227,19 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple } /** {@inheritDoc} */ + @Override public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner) { + try { + CacheDataStore delegate0 = init0(true); + + if (delegate0 != null) + delegate0.setRowCacheCleaner(rowCacheCleaner); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ @Override public void update( GridCacheContext cctx, KeyCacheObject key, http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java index ad2f731..ee7c255 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; /** * Data store for H2 rows. @@ -43,6 +44,9 @@ public class RowStore { /** */ private final boolean persistenceEnabled; + /** Row cache cleaner. */ + private GridQueryRowCacheCleaner rowCacheCleaner; + /** * @param grp Cache group. * @param freeList Free list. @@ -67,6 +71,9 @@ public class RowStore { public void removeRow(long link) throws IgniteCheckedException { assert link != 0; + if (rowCacheCleaner != null) + rowCacheCleaner.remove(link); + if (!persistenceEnabled) freeList.removeDataRowByLink(link); else { @@ -109,6 +116,9 @@ public class RowStore { public boolean updateRow(long link, CacheDataRow row) throws IgniteCheckedException { assert !persistenceEnabled || ctx.database().checkpointLockIsHeldByThread(); + if (rowCacheCleaner != null) + rowCacheCleaner.remove(link); + return freeList.updateDataRow(link, row); } @@ -118,4 +128,13 @@ public class RowStore { public FreeList freeList() { return freeList; } + + /** + * Inject rows cache cleaner. + * + * @param rowCacheCleaner Rows cache cleaner. + */ + public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner) { + this.rowCacheCleaner = rowCacheCleaner; + } } http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java index 044ce22..fc8ae11 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java @@ -61,9 +61,11 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockStateChecker; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingPageIO; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; @@ -1188,10 +1190,22 @@ public class PageMemoryImpl implements PageMemoryEx { /** * @param absPtr Absolute pointer to read lock. + * @param fullId Full page ID. * @param force Force flag. * @return Pointer to the page read buffer. */ private long readLockPage(long absPtr, FullPageId fullId, boolean force) { + return readLockPage(absPtr, fullId, force, true); + } + + /** + * @param absPtr Absolute pointer to read lock. + * @param fullId Full page ID. + * @param force Force flag. + * @param touch Update page timestamp. + * @return Pointer to the page read buffer. + */ + private long readLockPage(long absPtr, FullPageId fullId, boolean force, boolean touch) { int tag = force ? -1 : PageIdUtils.tag(fullId.pageId()); boolean locked = rwLock.readLock(absPtr + PAGE_LOCK_OFFSET, tag); @@ -1199,7 +1213,8 @@ public class PageMemoryImpl implements PageMemoryEx { if (!locked) return 0; - PageHeader.writeTimestamp(absPtr, U.currentTimeMillis()); + if (touch) + PageHeader.writeTimestamp(absPtr, U.currentTimeMillis()); assert GridUnsafe.getInt(absPtr + PAGE_OVERHEAD + 4) == 0; //TODO GG-11480 @@ -1845,6 +1860,8 @@ public class PageMemoryImpl implements PageMemoryEx { Collection cpPages = segCheckpointPages; + clearRowCache(fullPageId, absPtr); + if (isDirty(absPtr)) { // Can evict a dirty page only if should be written by a checkpoint. // These pages does not have tmp buffer. @@ -1880,6 +1897,44 @@ public class PageMemoryImpl implements PageMemoryEx { } /** + * @param fullPageId Full page ID to remove all links placed on the page from row cache. + * @param absPtr Absolute pointer of the page to evict. + * @throws IgniteCheckedException On error. + */ + private void clearRowCache(FullPageId fullPageId, long absPtr) throws IgniteCheckedException { + assert writeLock().isHeldByCurrentThread(); + + if (ctx.kernalContext().query() == null || !ctx.kernalContext().query().moduleEnabled()) + return; + + long pageAddr = readLockPage(absPtr, fullPageId, true, false); + + try { + if (PageIO.getType(pageAddr) != PageIO.T_DATA) + return; + + final GridQueryRowCacheCleaner cleaner = ctx.kernalContext().query() + .getIndexing().rowCacheCleaner(fullPageId.groupId()); + + if (cleaner == null) + return; + + DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr); + + io.forAllItems(pageAddr, new DataPageIO.CC() { + @Override public Void apply(long link) { + cleaner.remove(link); + + return null; + } + }); + } + finally { + readUnlockPage(absPtr); + } + } + + /** * Evict random oldest page from memory to storage. * * @return Relative address for evicted page. http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/query/CacheQueryObjectValueContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/CacheQueryObjectValueContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/CacheQueryObjectValueContext.java index 08f71c8..3e8758e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/CacheQueryObjectValueContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/CacheQueryObjectValueContext.java @@ -44,12 +44,12 @@ public class CacheQueryObjectValueContext implements CacheObjectValueContext { /** {@inheritDoc} */ @Override public boolean copyOnGet() { - return true; + return false; } /** {@inheritDoc} */ @Override public boolean storeValue() { - return false; + return true; } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java index 2a34bfc..9624244 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java @@ -209,11 +209,11 @@ public interface GridQueryIndexing { /** * Unregisters cache. * - * @param cacheName Cache name. + * @param cctx Cache context. * @param rmvIdx If {@code true}, will remove index. * @throws IgniteCheckedException If failed to drop cache schema. */ - public void unregisterCache(String cacheName, boolean rmvIdx) throws IgniteCheckedException; + public void unregisterCache(GridCacheContext cctx, boolean rmvIdx) throws IgniteCheckedException; /** * Registers type if it was not known before or updates it otherwise. @@ -325,4 +325,12 @@ public interface GridQueryIndexing { * @return {@code True} if insert. */ public boolean isInsertStatement(PreparedStatement nativeStmt); + + /** + * Return row cache cleaner. + * + * @param cacheGroupId Cache group id. + * @return Row cache cleaner. + */ + public GridQueryRowCacheCleaner rowCacheCleaner(int cacheGroupId); } http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index ab84a1a..f53d4e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -876,7 +876,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { return; try { - onCacheStop0(cctx.name(), removeIdx); + onCacheStop0(cctx, removeIdx); } finally { busyLock.leaveBusy(); @@ -1627,7 +1627,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { cacheNames.add(CU.mask(cacheName)); } catch (IgniteCheckedException | RuntimeException e) { - onCacheStop0(cacheName, true); + onCacheStop0(cctx, true); throw e; } @@ -1638,13 +1638,15 @@ public class GridQueryProcessor extends GridProcessorAdapter { * Unregister cache.

* Use with {@link #busyLock} where appropriate. * - * @param cacheName Cache name. - * @param rmvIdx If {@code true}, will remove index. + * @param cctx Cache context. + * @param destroy Destroy flag. */ - public void onCacheStop0(String cacheName, boolean rmvIdx) { + public void onCacheStop0(GridCacheContext cctx, boolean destroy) { if (idx == null) return; + String cacheName = cctx.name(); + synchronized (stateMux) { // Clear types. Iterator> it = types.entrySet().iterator(); @@ -1679,7 +1681,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { // Notify indexing. try { - idx.unregisterCache(cacheName, rmvIdx); + idx.unregisterCache(cctx, destroy); } catch (Exception e) { U.error(log, "Failed to clear indexing on cache unregister (will ignore): " + cacheName, e); http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java new file mode 100644 index 0000000..53f665f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query; + +/** + * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache. + */ +public interface GridQueryRowCacheCleaner { + /** + * Remove row by link. + * + * @param link Link to remove. + */ + void remove(long link); +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java index 4b93a46..3f769d7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryIndexing; import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.processors.query.QueryField; @@ -305,7 +306,7 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT } /** {@inheritDoc} */ - @Override public void unregisterCache(String spaceName, boolean rmvIdx) throws IgniteCheckedException { + @Override public void unregisterCache(GridCacheContext cctx, boolean rmvIdx) throws IgniteCheckedException { // No-op } @@ -375,5 +376,10 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT @Override public boolean isInsertStatement(PreparedStatement nativeStmt) { return false; } + + /** {@inheritDoc} */ + @Override public GridQueryRowCacheCleaner rowCacheCleaner(int cacheGroupId) { + return null; + } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java new file mode 100644 index 0000000..2c3a95f --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.PageMemory; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap; +import org.apache.ignite.internal.util.typedef.F; + +/** + * H2 row cache. + */ +public class H2RowCache implements GridQueryRowCacheCleaner { + /** Cached rows. */ + private ConcurrentHashMap rows = new ConcurrentHashMap<>(); + + /** Cache group ID. */ + private final CacheGroupContext grpCtx; + + /** Usage count. */ + private int usageCnt = 1; + + /** + * @param grpCtx Cache group context. + */ + public H2RowCache(CacheGroupContext grpCtx) { + this.grpCtx = grpCtx; + } + + /** + * Get row by link. + * + * @param link Link. + * @return Cached on-heap row. + * @throws IgniteCheckedException On error. + */ + public GridH2KeyValueRowOnheap get(long link) throws IgniteCheckedException { + GridH2KeyValueRowOnheap row = rows.get(link); + + if (row != null) + touch(link); + + return row; + } + + /** + * Put row by link. + * + * @param row Row. + */ + public void put(GridH2KeyValueRowOnheap row) { + rows.put(row.link(), row); + } + + /** {@inheritDoc} */ + @Override public void remove(long link) { + rows.remove(link); + } + + /** + * Cache registration callback. + */ + public void onCacheRegistered() { + usageCnt++; + } + + /** + * Cache un-registration callback. + * + * @param cctx Cache context. + * @return {@code True} if there are no more usages for the given cache group. + */ + public boolean onCacheUnregistered(GridCacheContext cctx) { + boolean res = --usageCnt == 0; + + clearForCache(cctx); + + return res; + } + + /** + * @return Cached rows count. + */ + public int size() { + return rows.size(); + } + + /** + * Clear entries belonging to the given cache. + * + * @param cctx Cache context. + */ + private void clearForCache(GridCacheContext cctx) { + int cacheId = cctx.cacheId(); + + Iterator> iter = rows.entrySet().iterator(); + + while (iter.hasNext()) { + GridH2KeyValueRowOnheap row = iter.next().getValue(); + + if (F.eq(cacheId, row.cacheId())) + iter.remove(); + } + } + + /** + * Update page + * + * @param link Link. + * @throws IgniteCheckedException On error. + */ + private void touch(long link) throws IgniteCheckedException { + PageMemory mem = grpCtx.dataRegion().pageMemory(); + + int grpId = grpCtx.groupId(); + + final long pageId = PageIdUtils.pageId(link); + + final long page = mem.acquirePage(grpId, pageId); + + try { + // Touch page timestamp + mem.readLock(grpId, pageId, page); + + mem.readUnlock(grpId, pageId, page); + } + finally { + mem.releasePage(grpId, pageId, page); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java new file mode 100644 index 0000000..9b1a03c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.jetbrains.annotations.Nullable; + +import java.util.HashMap; +import java.util.Map; + +/** + * H2 row cache registry. + */ +public class H2RowCacheRegistry { + /** Mutex. */ + private final Object mux = new Object(); + + /** Row caches for specific cache groups. */ + private volatile Map caches; + + /** + * Get row cache for the given cache group. + * + * @param grpId Cache group ID. + * @return Row cache or {@code null} if none available. + */ + @Nullable public H2RowCache forGroup(int grpId) { + return caches != null ? caches.get(grpId) : null; + } + + /** + * Callback invoked on cache registration within indexing. + * + * @param cctx Cache context. + */ + public void onCacheRegistered(GridCacheContext cctx) { + if (!cctx.config().isSqlOnheapCacheEnabled()) + return; + + synchronized (mux) { + int grpId = cctx.groupId(); + + if (caches != null) { + H2RowCache cache = caches.get(grpId); + + if (cache != null) { + cache.onCacheRegistered(); + + return; + } + } + + HashMap caches0 = copy(); + + H2RowCache rowCache = new H2RowCache(cctx.group()); + + caches0.put(grpId, rowCache); + + caches = caches0; + + // Inject row cache cleaner into store on cache creation. + // Used in case the cache with enabled SqlOnheapCache is created in exists cache group + // and SqlOnheapCache is disbaled for the caches have been created before. + for (IgniteCacheOffheapManager.CacheDataStore ds : cctx.offheap().cacheDataStores()) + ds.setRowCacheCleaner(rowCache); + } + } + + /** + * Callback invoked when cache gets unregistered. + * + * @param cctx Cache context. + */ + public void onCacheUnregistered(GridCacheContext cctx) { + if (!cctx.config().isSqlOnheapCacheEnabled()) + return; + + synchronized (mux) { + int grpId = cctx.groupId(); + + assert caches != null; + + H2RowCache cache = caches.get(grpId); + + assert cache != null; + + if (cache.onCacheUnregistered(cctx)) { + HashMap caches0 = copy(); + + caches0.remove(grpId); + + caches = caches0; + } + } + } + + /** + * Create copy of caches map under lock. + * + * @return Copy. + */ + private HashMap copy() { + assert Thread.holdsLock(mux); + + if (caches == null) + return new HashMap<>(); + else + return new HashMap<>(caches); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 7c451a5..4393946 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java @@ -84,6 +84,7 @@ import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter; import org.apache.ignite.internal.processors.query.GridQueryIndexing; +import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.GridRunningQueryInfo; import org.apache.ignite.internal.processors.query.IgniteSQLException; @@ -275,6 +276,9 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** */ private final ConcurrentMap runs = new ConcurrentHashMap8<>(); + /** Row cache. */ + private final H2RowCacheRegistry rowCache = new H2RowCacheRegistry(); + /** */ private final ThreadLocal connCache = new ThreadLocal() { @Nullable @Override public H2ConnectionWrapper get() { @@ -826,7 +830,9 @@ public class IgniteH2Indexing implements GridQueryIndexing { final int segments = tbl.rowDescriptor().context().config().getQueryParallelism(); - return new H2TreeIndex(cctx, tbl, name, pk, cols, inlineSize, segments); + H2RowCache cache = rowCache.forGroup(cctx.groupId()); + + return new H2TreeIndex(cctx, cache, tbl, name, pk, cols, inlineSize, segments); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -2043,6 +2049,11 @@ public class IgniteH2Indexing implements GridQueryIndexing { return prep instanceof Insert; } + /** {@inheritDoc} */ + @Override public GridQueryRowCacheCleaner rowCacheCleaner(int grpId) { + return rowCache.forGroup(grpId); + } + /** * Called periodically by {@link GridTimeoutProcessor} to clean up the {@link #stmtCache}. */ @@ -2421,6 +2432,8 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** {@inheritDoc} */ @Override public void registerCache(String cacheName, String schemaName, GridCacheContext cctx) throws IgniteCheckedException { + rowCache.onCacheRegistered(cctx); + if (!isDefaultSchema(schemaName)) { synchronized (schemaMux) { H2Schema schema = new H2Schema(schemaName); @@ -2442,7 +2455,11 @@ public class IgniteH2Indexing implements GridQueryIndexing { } /** {@inheritDoc} */ - @Override public void unregisterCache(String cacheName, boolean rmvIdx) { + @Override public void unregisterCache(GridCacheContext cctx, boolean rmvIdx) { + rowCache.onCacheUnregistered(cctx); + + String cacheName = cctx.name(); + String schemaName = schema(cacheName); H2Schema schema = schemas.get(schemaName); http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java index 67bde69..8da3b05 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java @@ -28,15 +28,18 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO; import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList; +import org.apache.ignite.internal.processors.query.h2.H2RowCache; import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasInnerIO; import org.apache.ignite.internal.processors.query.h2.database.io.H2ExtrasLeafIO; import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; import org.h2.result.SearchRow; import org.h2.table.IndexColumn; import org.h2.value.Value; +import org.jetbrains.annotations.Nullable; /** */ @@ -63,7 +66,12 @@ public abstract class H2Tree extends BPlusTree { } }; + /** Row cache. */ + private final H2RowCache rowCache; + /** + * Constructor. + * * @param name Tree name. * @param reuseList Reuse list. * @param grpId Cache group ID. @@ -72,6 +80,7 @@ public abstract class H2Tree extends BPlusTree { * @param rowStore Row data store. * @param metaPageId Meta page ID. * @param initNew Initialize new index. + * @param rowCache Row cache. * @throws IgniteCheckedException If failed. */ protected H2Tree( @@ -86,7 +95,8 @@ public abstract class H2Tree extends BPlusTree { boolean initNew, IndexColumn[] cols, List inlineIdxs, - int inlineSize + int inlineSize, + @Nullable H2RowCache rowCache ) throws IgniteCheckedException { super(name, grpId, pageMem, wal, globalRmvId, metaPageId, reuseList); @@ -108,16 +118,35 @@ public abstract class H2Tree extends BPlusTree { for (int i = 0; i < cols.length; i++) columnIds[i] = cols[i].column.getColumnId(); + this.rowCache = rowCache; + setIos(H2ExtrasInnerIO.getVersions(inlineSize), H2ExtrasLeafIO.getVersions(inlineSize)); initTree(initNew, inlineSize); } /** - * @return Row store. + * Create row from link. + * + * @param link Link. + * @return Row. + * @throws IgniteCheckedException if failed. */ - public H2RowFactory getRowFactory() { - return rowStore; + public GridH2Row createRowFromLink(long link) throws IgniteCheckedException { + if (rowCache != null) { + GridH2Row row = rowCache.get(link); + + if (row == null) { + row = rowStore.getRow(link); + + if (row instanceof GridH2KeyValueRowOnheap) + rowCache.put((GridH2KeyValueRowOnheap)row); + } + + return row; + } + else + return rowStore.getRow(link); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java index 18d9bec..c1336eff 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.h2.H2Cursor; +import org.apache.ignite.internal.processors.query.h2.H2RowCache; import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; @@ -72,7 +73,7 @@ public class H2TreeIndex extends GridH2IndexBase { private final List inlineIdxs; /** Cache context. */ - private GridCacheContext cctx; + private final GridCacheContext cctx; /** * @param cctx Cache context. @@ -85,6 +86,7 @@ public class H2TreeIndex extends GridH2IndexBase { */ public H2TreeIndex( GridCacheContext cctx, + @Nullable H2RowCache rowCache, GridH2Table tbl, String name, boolean pk, @@ -95,6 +97,7 @@ public class H2TreeIndex extends GridH2IndexBase { assert segmentsCnt > 0 : segmentsCnt; this.cctx = cctx; + IndexColumn[] cols = colsList.toArray(new IndexColumn[colsList.size()]); IndexColumn.mapColumns(cols, tbl); @@ -135,7 +138,8 @@ public class H2TreeIndex extends GridH2IndexBase { page.isAllocated(), cols, inlineIdxs, - computeInlineSize(inlineIdxs, inlineSize)) { + computeInlineSize(inlineIdxs, inlineSize), + rowCache) { @Override public int compareValues(Value v1, Value v2) { return v1 == v2 ? 0 : table.compareTypeSafe(v1, v2); } http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java index a4aa600..294492d 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java @@ -111,7 +111,7 @@ public class H2ExtrasInnerIO extends BPlusInnerIO implements H2RowLin assert link != 0; - return ((H2Tree)tree).getRowFactory().getRow(link); + return ((H2Tree)tree).createRowFromLink(link); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java index 8723601..4770295 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java @@ -123,7 +123,7 @@ public class H2ExtrasLeafIO extends BPlusLeafIO implements H2RowLinkI throws IgniteCheckedException { long link = getLink(pageAddr, idx); - return ((H2Tree)tree).getRowFactory().getRow(link); + return ((H2Tree)tree).createRowFromLink(link); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java index a1f1ce9..cf37bb7 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java @@ -57,7 +57,7 @@ public class H2InnerIO extends BPlusInnerIO implements H2RowLinkIO { throws IgniteCheckedException { long link = getLink(pageAddr, idx); - return ((H2Tree)tree).getRowFactory().getRow(link); + return ((H2Tree)tree).createRowFromLink(link); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java index 85dcf50..55a980f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java @@ -64,7 +64,7 @@ public class H2LeafIO extends BPlusLeafIO implements H2RowLinkIO { throws IgniteCheckedException { long link = getLink(pageAddr, idx); - return ((H2Tree)tree).getRowFactory().getRow(link); + return ((H2Tree)tree).createRowFromLink(link); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCachePageEvictionTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCachePageEvictionTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCachePageEvictionTest.java new file mode 100644 index 0000000..ba5edc9 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCachePageEvictionTest.java @@ -0,0 +1,238 @@ +/* + * 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.index; + +import java.util.Collections; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataPageEvictionMode; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.h2.H2RowCache; +import org.apache.ignite.internal.processors.query.h2.H2RowCacheRegistry; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for H2RowCacheRegistry with page eviction. + */ +public class H2RowCachePageEvictionTest extends GridCommonAbstractTest { + /** Entries count. */ + private static final int ENTRIES = 10_000; + + /** Offheap size for memory policy. */ + private static final int SIZE = 12 * 1024 * 1024; + + /** Test time. */ + private static final int TEST_TIME = 3 * 60_000; + + /** Default policy name. */ + private static final String DATA_REGION_NAME = "default"; + + /** Default policy name. */ + private static final String CACHE_NAME = "cache"; + + /** Random generator. */ + private static final ThreadLocalRandom RND = ThreadLocalRandom.current(); + + /** Default policy name. */ + private static boolean persistenceEnabled; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", true); + U.resolveWorkDirectory(U.defaultWorkDirectory(), "wal", true); + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return TEST_TIME + 10 * 60_000; + } + + /** + * @param name Cache name. + * @param sqlOnheapCacheEnabled sqlOnheapCacheEnabled flag. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(String name, boolean sqlOnheapCacheEnabled) { + return new CacheConfiguration() + .setName(name) + .setSqlOnheapCacheEnabled(sqlOnheapCacheEnabled) + .setDataRegionName(DATA_REGION_NAME) + .setAffinity(new RendezvousAffinityFunction(false, 2)) + .setQueryEntities(Collections.singleton( + new QueryEntity(Integer.class, Value.class))); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setPersistenceEnabled(persistenceEnabled) + .setMaxSize(SIZE) + .setInitialSize(SIZE) + .setPageEvictionMode(persistenceEnabled ? DataPageEvictionMode.DISABLED + : DataPageEvictionMode.RANDOM_LRU) + .setName(DATA_REGION_NAME))); + + return cfg; + } + + /** + */ + private void checkRowCacheOnPageEviction() { + grid().getOrCreateCache(cacheConfiguration(CACHE_NAME, true)); + + int grpId = grid().cachex(CACHE_NAME).context().groupId(); + + assertEquals(grpId, grid().cachex(CACHE_NAME).context().groupId()); + + try (IgniteDataStreamer stream = grid().dataStreamer(CACHE_NAME)) { + for (int i = 0; i < ENTRIES; ++i) + stream.addData(i, new Value(i)); + } + + H2RowCache rowCache = rowCache(grid()).forGroup(grpId); + + fillRowCache(CACHE_NAME); + + assertNotNull(rowCache); + + int rowCacheSizeBeforeEvict = rowCache.size(); + + try (IgniteDataStreamer stream = grid().dataStreamer(CACHE_NAME)) { + for (int i = ENTRIES; i < 2 * ENTRIES; ++i) + stream.addData(i, new Value(i)); + } + + assertTrue("rowCache size before evictions: " + rowCacheSizeBeforeEvict + + ", after evictions: " + rowCache.size(), + rowCacheSizeBeforeEvict > rowCache.size()); + } + + /** + * @throws Exception On error. + */ + public void testEvictPagesWithDiskStorageSingleCacheInGroup() throws Exception { + persistenceEnabled = true; + + startGrid(); + + grid().active(true); + + checkRowCacheOnPageEviction(); + } + + /** + * @throws Exception On error. + */ + public void testEvictPagesWithDiskStorageWithOtherCacheInGroup() throws Exception { + persistenceEnabled = true; + + startGrid(); + + grid().active(true); + + grid().getOrCreateCache(cacheConfiguration("cacheWithoutOnHeapCache", false)); + + checkRowCacheOnPageEviction(); + } + + /** + * @throws Exception On error. + */ + public void testEvictPagesWithoutDiskStorageSingleCacheInGroup() throws Exception { + persistenceEnabled = false; + + startGrid(); + + checkRowCacheOnPageEviction(); + } + + /** + * @throws Exception On error. + */ + public void testEvictPagesWithoutDiskStorageWithOtherCacheInGroup() throws Exception { + persistenceEnabled = false; + + startGrid(); + + grid().getOrCreateCache(cacheConfiguration("cacheWithoutOnHeapCache", false)); + + checkRowCacheOnPageEviction(); + } + + /** + * @param ig Ignite node. + * @return H2RowCacheRegistry for checks. + */ + private H2RowCacheRegistry rowCache(IgniteEx ig) { + IgniteH2Indexing indexing = (IgniteH2Indexing)ig.context().query().getIndexing(); + + return GridTestUtils.getFieldValue(indexing, "rowCache"); + } + + /** + * @param name Cache name. + */ + @SuppressWarnings("unchecked") + private void fillRowCache(String name) { + for (int i = 0; i < ENTRIES; ++i) + grid().cache(name).query(new SqlQuery(Value.class, "_key = " + i)).getAll(); + } + + /** + * + */ + private static class Value { + /** Long value. */ + @QuerySqlField + private long lVal; + + /** String value. */ + @QuerySqlField + private byte bytes[] = new byte[1024]; + + /** + * @param k Key. + */ + Value(int k) { + lVal = k; + RND.nextBytes(bytes); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCacheSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCacheSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCacheSelfTest.java new file mode 100644 index 0000000..f3bb719 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2RowCacheSelfTest.java @@ -0,0 +1,352 @@ +/* + * 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.index; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import javax.cache.Cache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.h2.H2RowCache; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests H2RowCacheRegistry. + */ +public class H2RowCacheSelfTest extends GridCommonAbstractTest { + /** Keys count. */ + private static final int ENTRIES = 1_000; + + /** Random generator. */ + private static final Random RND = new Random(System.currentTimeMillis()); + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + startGrid(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @param name Cache name. + * @param enableOnheapCache Enable on-heal SQL rows cache. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(String name, boolean enableOnheapCache) { + return new CacheConfiguration() + .setName(name) + .setSqlOnheapCacheEnabled(enableOnheapCache) + .setGroupName("group") + .setQueryEntities(Collections.singleton( + new QueryEntity(Integer.class, Value.class))); + } + + /** + */ + public void testDestroyCacheCreation() { + final String cacheName0 = "cache0"; + final String cacheName1 = "cache1"; + + grid().getOrCreateCache(cacheConfiguration(cacheName0, false)); + + int grpId = grid().cachex(cacheName0).context().groupId(); + + assertNull(rowCache(grid(), grpId)); + + grid().getOrCreateCache(cacheConfiguration(cacheName1, true)); + + assertEquals(grpId, grid().cachex(cacheName1).context().groupId()); + + assertNotNull(rowCache(grid(), grpId)); + } + + /** + * @throws IgniteCheckedException If failed. + */ + public void testDestroyCacheSingleCacheInGroup() throws IgniteCheckedException { + checkDestroyCache(); + } + + /** + * @throws IgniteCheckedException If failed. + */ + public void testDestroyCacheWithOtherCacheInGroup() throws IgniteCheckedException { + grid().getOrCreateCache(cacheConfiguration("cacheWithoutOnheapCache", false)); + + checkDestroyCache(); + } + + /** + * @throws Exception If failed. + */ + public void testDeleteEntryCacheSingleCacheInGroup() throws Exception { + checkDeleteEntry(); + } + + /** + * @throws Exception If failed. + */ + public void testDeleteEntryWithOtherCacheInGroup() throws Exception { + grid().getOrCreateCache(cacheConfiguration("cacheWithoutOnheapCache", false)); + + checkDeleteEntry(); + } + + /** + * @throws Exception If failed. + */ + public void testUpdateEntryCacheSingleCacheInGroup() throws Exception { + checkDeleteEntry(); + } + + /** + * @throws Exception If failed. + */ + public void testUpdateEntryWithOtherCacheInGroup() throws Exception { + grid().getOrCreateCache(cacheConfiguration("cacheWithoutOnheapCache", false)); + + checkUpdateEntry(); + } + + /** + * @throws IgniteCheckedException If failed. + */ + private void checkDestroyCache() throws IgniteCheckedException { + final String cacheName0 = "cache0"; + final String cacheName1 = "cache1"; + + grid().getOrCreateCache(cacheConfiguration(cacheName0, true)); + grid().getOrCreateCache(cacheConfiguration(cacheName1, true)); + + int grpId = grid().cachex(cacheName0).context().groupId(); + + assertEquals(grpId, grid().cachex(cacheName1).context().groupId()); + + try(IgniteDataStreamer streamer = grid().dataStreamer(cacheName0)) { + for (int i = 0; i < ENTRIES / 2; ++i) + streamer.addData(i, new Value(i)); + } + + try(IgniteDataStreamer streamer = grid().dataStreamer(cacheName1)) { + for (int i = ENTRIES / 2; i < ENTRIES; ++i) + streamer.addData(i, new Value(i)); + } + + H2RowCache rowCache = rowCache(grid(), grpId); + + assertNotNull(rowCache); + + Set linksOfCache0 = new HashSet<>(ENTRIES / 2); + Set linksOfCache1 = new HashSet<>(ENTRIES / 2); + + for (int i = 0; i < ENTRIES / 2; ++i) + linksOfCache0.add(getLinkForKey(cacheName0, rowCache(grid(), grpId), i)); + + for (int i = ENTRIES / 2; i < ENTRIES; ++i) + linksOfCache1.add(getLinkForKey(cacheName1, rowCache(grid(), grpId), i)); + + grid().destroyCache(cacheName0); + + assertNotNull(rowCache(grid(), grpId)); + + for (long link : linksOfCache0) + assertNull(rowCache(grid(), grpId).get(link)); + + grid().destroyCache(cacheName1); + + assertNull(rowCache(grid(), grpId)); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void checkDeleteEntry() throws Exception { + final String cacheName = "cache"; + + grid().getOrCreateCache(cacheConfiguration(cacheName, true)); + + int grpId = grid().cachex(cacheName).context().groupId(); + + assertEquals(grpId, grid().cachex(cacheName).context().groupId()); + + fillCache(cacheName); + + H2RowCache rowCache = rowCache(grid(), grpId); + + fillRowCache(cacheName); + + assertNotNull(rowCache); + + int key = RND.nextInt(ENTRIES); + + grid().cache(cacheName) + .query(new SqlQuery(Value.class, "_key = " + key)).getAll(); + + int rowCacheSize = rowCache.size(); + + long rowLink = getLinkForKey(cacheName, rowCache, key); + + assertNotNull(rowCache.get(rowLink)); + + // Remove + grid().cache(cacheName).remove(key); + + assertNull(rowCache.get(rowLink)); + + int rowCacheSizeAfterUpdate = rowCache.size(); + + assertEquals(rowCacheSize - 1, rowCacheSizeAfterUpdate); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void checkUpdateEntry() throws Exception { + final String cacheName = "cache"; + + grid().getOrCreateCache(cacheConfiguration(cacheName, true)); + + int grpId = grid().cachex(cacheName).context().groupId(); + + assertEquals(grpId, grid().cachex(cacheName).context().groupId()); + + fillCache(cacheName); + + H2RowCache rowCache = rowCache(grid(), grpId); + + fillRowCache(cacheName); + + assertNotNull(rowCache); + + int key = RND.nextInt(ENTRIES); + + long rowLink = getLinkForKey(cacheName, rowCache, key); + + int rowCacheSize = rowCache.size(); + + assertNotNull(rowCache.get(rowLink)); + + // Update row + grid().cache(cacheName).put(key, new Value(key + 1)); + + assertNull(rowCache.get(rowLink)); + + int rowCacheSizeAfterUpdate = rowCache.size(); + + assertEquals(rowCacheSize - 1, rowCacheSizeAfterUpdate); + + // Check updated value. + List> res = grid().cache(cacheName) + .query(new SqlQuery(Value.class, "_key = " + key)).getAll(); + + assertEquals(1, res.size()); + assertEquals(key + 1, (int)res.get(0).getValue().lVal); + } + + /** + * @param cacheName Cache name. + * @param rowCache Row cache. + * @param key Key to find. + * @return Row's link. + */ + private long getLinkForKey(String cacheName, H2RowCache rowCache, int key) { + grid().cache(cacheName) + .query(new SqlQuery(Value.class, "_key = " + key)).getAll().size(); + + ConcurrentHashMap rowsMap = GridTestUtils.getFieldValue(rowCache, "rows"); + + for (Map.Entry e : rowsMap.entrySet()) { + GridH2KeyValueRowOnheap val = e.getValue(); + + if ((Integer)val.key().value(null, false) == key) + return e.getKey(); + } + + fail("Row cache doesn't contain key [key=" + key + ']'); + + return -1; + } + + /** + * @param ig Ignite node. + * @param grpId Cache group ID. + * @return H2RowCache for checks. + */ + private H2RowCache rowCache(IgniteEx ig, int grpId) { + return (H2RowCache)ig.context().query().getIndexing().rowCacheCleaner(grpId); + } + + /** + * @param name Cache name. + */ + private void fillCache(String name) { + try(IgniteDataStreamer streamer = grid().dataStreamer(name)) { + for (int i = 0; i < ENTRIES; ++i) + streamer.addData(i, new Value(i)); + } + } + + /** + * @param name Cache name. + */ + @SuppressWarnings("unchecked") + private void fillRowCache(String name) { + for (int i = 0; i < ENTRIES; ++i) + grid().cache(name).query(new SqlQuery(Value.class, "_key = " + i)).getAll().size(); + } + + /** + * + */ + private static class Value { + /** Long value. */ + @QuerySqlField + private long lVal; + + /** String value. */ + @QuerySqlField + private String strVal; + + /** + * @param k Key. + */ + Value(int k) { + lVal = k; + strVal = "val_" + k; + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java index db732eb..8e3dc5c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java @@ -51,7 +51,6 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; */ public class OptimizedMarshallerIndexNameTest extends GridCommonAbstractTest { - /** Test name 1 */ private static final String TEST_NAME1 = "Name1"; /** Test name 2 */ @@ -102,13 +101,19 @@ public class OptimizedMarshallerIndexNameTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), workSubdir(), true)); startGrid(getTestIgniteInstanceName()); grid().active(true); } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + /** * Verifies that BPlusTree are not erroneously shared between tables in the same cache * due to IGNITE-6915 bug. http://git-wip-us.apache.org/repos/asf/ignite/blob/693a9593/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java index 564019a..cf5c9a6 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java @@ -122,6 +122,8 @@ import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComple import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerTransactionalReplicatedTest; import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest; import org.apache.ignite.internal.processors.cache.index.H2ConnectionLeaksSelfTest; +import org.apache.ignite.internal.processors.cache.index.H2RowCachePageEvictionTest; +import org.apache.ignite.internal.processors.cache.index.H2RowCacheSelfTest; import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest; import org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest; import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest; @@ -238,7 +240,6 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite { suite.addTestSuite(IgniteCacheLargeResultSelfTest.class); suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class); suite.addTestSuite(IgniteSqlBigIntegerKeyTest.class); - suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class); suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class); @@ -379,6 +380,10 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite { suite.addTestSuite(IgniteSqlDefaultValueTest.class); + // H2 Rows on-heap cache + suite.addTestSuite(H2RowCacheSelfTest.class); + suite.addTestSuite(H2RowCachePageEvictionTest.class); + return suite; } }