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 C0B07200D20 for ; Tue, 17 Oct 2017 16:07:15 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id BE8DC1609EB; Tue, 17 Oct 2017 14:07:15 +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 E24011609DE for ; Tue, 17 Oct 2017 16:07:13 +0200 (CEST) Received: (qmail 2769 invoked by uid 500); 17 Oct 2017 14:07:13 -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 2760 invoked by uid 99); 17 Oct 2017 14:07:13 -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; Tue, 17 Oct 2017 14:07:13 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id EF648DFAB2; Tue, 17 Oct 2017 14:07:12 +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 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: Changed internal indexing signatures for store/remove, fixed error in SchemaIndexCacheVisitorImpl (link can not be used without entry lock). Date: Tue, 17 Oct 2017 14:07:12 +0000 (UTC) archived-at: Tue, 17 Oct 2017 14:07:15 -0000 Repository: ignite Updated Branches: refs/heads/master cc048f082 -> 879bf581b Changed internal indexing signatures for store/remove, fixed error in SchemaIndexCacheVisitorImpl (link can not be used without entry lock). Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/879bf581 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/879bf581 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/879bf581 Branch: refs/heads/master Commit: 879bf581b323d961a88a3a6a3f26a4ab26638739 Parents: cc048f0 Author: sboikov Authored: Tue Oct 17 17:06:56 2017 +0300 Committer: sboikov Committed: Tue Oct 17 17:06:56 2017 +0300 ---------------------------------------------------------------------- .../processors/cache/GridCacheEntryEx.java | 3 +- .../processors/cache/GridCacheMapEntry.java | 8 +- .../cache/IgniteCacheOffheapManagerImpl.java | 42 +--------- .../cache/query/GridCacheQueryManager.java | 53 ++++-------- .../processors/query/GridQueryIndexing.java | 25 +++--- .../processors/query/GridQueryProcessor.java | 75 +++++++++-------- .../schema/SchemaIndexCacheVisitorClosure.java | 14 +--- .../schema/SchemaIndexCacheVisitorImpl.java | 29 ++++--- .../processors/cache/GridCacheTestEntryEx.java | 2 +- ...IgniteClientCacheInitializationFailTest.java | 12 ++- .../query/h2/DmlStatementsProcessor.java | 2 +- .../processors/query/h2/IgniteH2Indexing.java | 62 +++++++-------- .../query/h2/database/H2PkHashIndex.java | 5 +- .../query/h2/database/H2RowFactory.java | 6 +- .../processors/query/h2/database/H2Tree.java | 17 +++- .../query/h2/opt/GridH2KeyRowOnheap.java | 6 +- .../query/h2/opt/GridH2KeyValueRowOnheap.java | 35 +++----- .../processors/query/h2/opt/GridH2Row.java | 59 +++++--------- .../query/h2/opt/GridH2RowDescriptor.java | 24 ++---- .../processors/query/h2/opt/GridH2Table.java | 84 ++++++++------------ .../cache/IgniteCacheAbstractQuerySelfTest.java | 13 +-- 21 files changed, 221 insertions(+), 355 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java index b2cabac..6da7bc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java @@ -870,11 +870,10 @@ public interface GridCacheEntryEx { * Update index from within entry lock, passing key, value, and expiration time to provided closure. * * @param clo Closure to apply to key, value, and expiration time. - * @param link Link. * @throws IgniteCheckedException If failed. * @throws GridCacheEntryRemovedException If entry was removed. */ - public void updateIndex(SchemaIndexCacheVisitorClosure clo, long link) throws IgniteCheckedException, + public void updateIndex(SchemaIndexCacheVisitorClosure clo) throws IgniteCheckedException, GridCacheEntryRemovedException; /** http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 54b8dc3..5c3fe1f 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 @@ -3300,7 +3300,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme } /** {@inheritDoc} */ - @Override public void updateIndex(SchemaIndexCacheVisitorClosure clo, long link) throws IgniteCheckedException, + @Override public void updateIndex(SchemaIndexCacheVisitorClosure clo) throws IgniteCheckedException, GridCacheEntryRemovedException { synchronized (this) { if (isInternal()) @@ -3308,10 +3308,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme checkObsolete(); - unswap(false); + CacheDataRow row = cctx.offheap().read(this); - if (val != null) - clo.apply(key, partition(), val, ver, expireTimeUnlocked(), link); + if (row != null) + clo.apply(row); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 ba6f7d0..4844686 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 @@ -1339,27 +1339,8 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; - if (qryMgr.enabled()) { - if (oldRow != null) { - qryMgr.store(key, - partId, - oldRow.value(), - oldRow.version(), - newRow.value(), - newRow.version(), - expireTime, - newRow.link()); - } - else { - qryMgr.store(key, - partId, - null, null, - newRow.value(), - newRow.version(), - expireTime, - newRow.link()); - } - } + if (qryMgr.enabled()) + qryMgr.store(newRow, oldRow); if (oldRow != null) { assert oldRow.link() != 0 : oldRow; @@ -1391,15 +1372,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager GridCacheQueryManager qryMgr = cctx.queries(); - qryMgr.store( - key, - partId, - null, - null, - row.value(), - row.version(), - row.expireTime(), - row.link()); + qryMgr.store(row, null); } } @@ -1427,9 +1400,6 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager * @throws IgniteCheckedException If failed. */ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable CacheDataRow oldRow) throws IgniteCheckedException { - CacheObject val = null; - GridCacheVersion ver = null; - if (oldRow != null) { int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID; @@ -1441,16 +1411,12 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager pendingEntries.removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); decrementSize(cctx.cacheId()); - - val = oldRow.value(); - - ver = oldRow.version(); } GridCacheQueryManager qryMgr = cctx.queries(); if (qryMgr.enabled()) - qryMgr.remove(key, partId, val, ver); + qryMgr.remove(key, oldRow); if (oldRow != null) rowStore.removeRow(oldRow.link()); http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java index 392b19f..dc4d7e0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java @@ -379,33 +379,14 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte } /** - * Writes key-value pair to index. - * - * @param key Key. - * @param partId Partition. - * @param prevVal Previous value. - * @param prevVer Previous version. - * @param val Value. - * @param ver Cache entry version. - * @param expirationTime Expiration time or 0 if never expires. - * @param link Link. + * @param newRow New row. + * @param prevRow Previous row. * @throws IgniteCheckedException In case of error. */ - public void store(KeyCacheObject key, - int partId, - @Nullable CacheObject prevVal, - @Nullable GridCacheVersion prevVer, - CacheObject val, - GridCacheVersion ver, - long expirationTime, - long link) + public void store(CacheDataRow newRow, @Nullable CacheDataRow prevRow) throws IgniteCheckedException { - assert key != null; - assert val != null; assert enabled(); - - if (key instanceof GridCacheInternal) - return; // No-op. + assert newRow != null && newRow.value() != null && newRow.link() != 0 : newRow; if (!enterBusy()) throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); @@ -414,15 +395,15 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte if (isIndexingSpiEnabled()) { CacheObjectContext coctx = cctx.cacheObjectContext(); - Object key0 = unwrapIfNeeded(key, coctx); + Object key0 = unwrapIfNeeded(newRow.key(), coctx); - Object val0 = unwrapIfNeeded(val, coctx); + Object val0 = unwrapIfNeeded(newRow.value(), coctx); - cctx.kernalContext().indexing().store(cacheName, key0, val0, expirationTime); + cctx.kernalContext().indexing().store(cacheName, key0, val0, newRow.expireTime()); } if (qryProcEnabled) - qryProc.store(cacheName, key, partId, prevVal, prevVer, val, ver, expirationTime, link); + qryProc.store(cctx, newRow, prevRow); } finally { invalidateResultCache(); @@ -433,17 +414,11 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte /** * @param key Key. - * @param partId Partition. - * @param val Value. - * @param ver Version. + * @param prevRow Previous row. * @throws IgniteCheckedException Thrown in case of any errors. */ - @SuppressWarnings("SimplifiableIfStatement") - public void remove(KeyCacheObject key, int partId, CacheObject val, - GridCacheVersion ver) throws IgniteCheckedException { - assert key != null; - - if (!QueryUtils.isEnabled(cctx.config()) && !(key instanceof GridCacheInternal)) + public void remove(KeyCacheObject key, @Nullable CacheDataRow prevRow) throws IgniteCheckedException { + if (!QueryUtils.isEnabled(cctx.config())) return; // No-op. if (!enterBusy()) @@ -457,8 +432,8 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte } // val may be null if we have no previous value. We should not call processor in this case. - if (qryProcEnabled && val != null) - qryProc.remove(cacheName, key, partId, val, ver); + if (qryProcEnabled && prevRow != null) + qryProc.remove(cctx, prevRow); } finally { invalidateResultCache(); @@ -474,7 +449,7 @@ public abstract class GridCacheQueryManager extends GridCacheManagerAdapte * @return Query future. */ @SuppressWarnings("unchecked") - public CacheQueryFuture queryLocal(GridCacheQueryBean qry) { + CacheQueryFuture queryLocal(GridCacheQueryBean qry) { assert qry.query().type() != GridCacheQueryType.SCAN : qry; if (log.isDebugEnabled()) http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 93d541d..b0a3831 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 @@ -30,10 +30,8 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.lang.GridCloseableIterator; @@ -216,27 +214,24 @@ public interface GridQueryIndexing { * Updates index. Note that key is unique for cache, so if cache contains multiple indexes * the key should be removed from indexes other than one being updated. * - * @param cacheName Cache name. + * @param cctx Cache context. * @param type Type descriptor. - * @param key Key. - * @param val Value. - * @param ver Version. - * @param expirationTime Expiration time or 0 if never expires. + * @param row New row. * @throws IgniteCheckedException If failed. */ - public void store(String cacheName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, CacheObject val, - GridCacheVersion ver, long expirationTime, long link) throws IgniteCheckedException; + public void store(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row) + throws IgniteCheckedException; /** * Removes index entry by key. * - * @param cacheName Cache name. - * @param key Key. - * @param val Value. + * @param cctx Cache context. + * @param type Type descriptor. + * @param row Row. * @throws IgniteCheckedException If failed. */ - public void remove(String cacheName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, CacheObject val, - GridCacheVersion ver) throws IgniteCheckedException; + public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row) + throws IgniteCheckedException; /** * Rebuilds all indexes of given type from hash index. http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 3a1cdb7..e88a234 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 @@ -67,11 +67,11 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl; @@ -1694,30 +1694,21 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** - * Writes key-value pair to index. - * - * @param cacheName Cache name. - * @param key Key. - * @param val Value. - * @param ver Cache entry version. - * @param expirationTime Expiration time or 0 if never expires. + * @param cctx Cache context. + * @param newRow New row. + * @param prevRow Previous row. * @throws IgniteCheckedException In case of error. */ @SuppressWarnings({"unchecked", "ConstantConditions"}) - public void store(final String cacheName, - final KeyCacheObject key, - int partId, - @Nullable CacheObject prevVal, - @Nullable GridCacheVersion prevVer, - final CacheObject val, - GridCacheVersion ver, - long expirationTime, - long link) throws IgniteCheckedException { - assert key != null; - assert val != null; + public void store(GridCacheContext cctx, CacheDataRow newRow, @Nullable CacheDataRow prevRow) + throws IgniteCheckedException { + assert cctx != null; + assert newRow != null; + + KeyCacheObject key = newRow.key(); if (log.isDebugEnabled()) - log.debug("Store [cache=" + cacheName + ", key=" + key + ", val=" + val + "]"); + log.debug("Store [cache=" + cctx.name() + ", key=" + key + ", val=" + newRow.value() + "]"); if (idx == null) return; @@ -1726,21 +1717,27 @@ public class GridQueryProcessor extends GridProcessorAdapter { throw new NodeStoppingException("Operation has been cancelled (node is stopping)."); try { - CacheObjectContext coctx = cacheObjectContext(cacheName); + String cacheName = cctx.name(); + + CacheObjectContext coctx = cctx.cacheObjectContext(); - QueryTypeDescriptorImpl desc = typeByValue(cacheName, coctx, key, val, true); + QueryTypeDescriptorImpl desc = typeByValue(cacheName, coctx, key, newRow.value(), true); - if (prevVal != null) { - QueryTypeDescriptorImpl prevValDesc = typeByValue(cacheName, coctx, key, prevVal, false); + if (prevRow != null) { + QueryTypeDescriptorImpl prevValDesc = typeByValue(cacheName, + coctx, + key, + prevRow.value(), + false); if (prevValDesc != null && prevValDesc != desc) - idx.remove(cacheName, prevValDesc, key, partId, prevVal, prevVer); + idx.remove(cctx, prevValDesc, prevRow); } if (desc == null) return; - idx.store(cacheName, desc, key, partId, val, ver, expirationTime, link); + idx.store(cctx, desc, newRow); } finally { busyLock.leaveBusy(); @@ -1870,7 +1867,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { try { final String schemaName = qry.getSchema() != null ? qry.getSchema() : idx.schema(cctx.name()); - final int mainCacheId = CU.cacheId(cctx.name()); + final int mainCacheId = cctx.cacheId(); IgniteOutClosureX>> clo; @@ -2056,7 +2053,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { try { final String schemaName = idx.schema(cctx.name()); - final int mainCacheId = CU.cacheId(cctx.name()); + final int mainCacheId = cctx.cacheId(); return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx, new IgniteOutClosureX>>() { @@ -2085,7 +2082,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { throw new IllegalStateException("Failed to execute query (grid is stopping)."); final String schemaName = idx.schema(cctx.name()); - final int mainCacheId = CU.cacheId(cctx.name()); + final int mainCacheId = cctx.cacheId(); try { return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx, @@ -2302,16 +2299,16 @@ public class GridQueryProcessor extends GridProcessorAdapter { } /** - * @param cacheName Cache name. - * @param key Key. + * @param cctx Cache context. + * @param val Row. * @throws IgniteCheckedException Thrown in case of any errors. */ - public void remove(String cacheName, KeyCacheObject key, int partId, CacheObject val, GridCacheVersion ver) + public void remove(GridCacheContext cctx, CacheDataRow val) throws IgniteCheckedException { - assert key != null; + assert val != null; if (log.isDebugEnabled()) - log.debug("Remove [cacheName=" + cacheName + ", key=" + key + ", val=" + val + "]"); + log.debug("Remove [cacheName=" + cctx.name() + ", key=" + val.key()+ ", val=" + val.value() + "]"); if (idx == null) return; @@ -2320,14 +2317,16 @@ public class GridQueryProcessor extends GridProcessorAdapter { throw new IllegalStateException("Failed to remove from index (grid is stopping)."); try { - CacheObjectContext coctx = cacheObjectContext(cacheName); - - QueryTypeDescriptorImpl desc = typeByValue(cacheName, coctx, key, val, false); + QueryTypeDescriptorImpl desc = typeByValue(cctx.name(), + cctx.cacheObjectContext(), + val.key(), + val.value(), + false); if (desc == null) return; - idx.remove(cacheName, desc, key, partId, val, ver); + idx.remove(cctx, desc, val); } finally { busyLock.leaveBusy(); http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorClosure.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorClosure.java index 7f50089..a934d69 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorClosure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorClosure.java @@ -18,9 +18,7 @@ package org.apache.ignite.internal.processors.query.schema; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.CacheObject; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; /** * Index closure accepting current entry state. @@ -29,14 +27,8 @@ public interface SchemaIndexCacheVisitorClosure { /** * Apply closure. * - * @param key Key. - * @param part Partition. - * @param val Value. - * @param ver Version. - * @param expiration Expiration. - * @param link Link. + * @param row Row. * @throws IgniteCheckedException If failed. */ - public void apply(KeyCacheObject key, int part, CacheObject val, GridCacheVersion ver, long expiration, long link) - throws IgniteCheckedException; + public void apply(CacheDataRow row) throws IgniteCheckedException; } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java index 4e50f64..fda7d1d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java @@ -17,24 +17,22 @@ package org.apache.ignite.internal.processors.query.schema; +import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; -import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.typedef.internal.S; -import java.util.Collection; - import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.EVICTED; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.RENTING; @@ -111,14 +109,17 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor { return; try { - GridCursor cursor = part.dataStore().cursor(); + GridCursor cursor = part.dataStore().cursor(cctx.cacheId(), + null, + null, + CacheDataRowAdapter.RowData.KEY_ONLY); while (cursor.next()) { CacheDataRow row = cursor.get(); KeyCacheObject key = row.key(); - processKey(key, row.link(), clo); + processKey(key, clo); if (part.state() == RENTING) break; @@ -133,11 +134,10 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor { * Process single key. * * @param key Key. - * @param link Link. * @param clo Closure. * @throws IgniteCheckedException If failed. */ - private void processKey(KeyCacheObject key, long link, FilteringVisitorClosure clo) throws IgniteCheckedException { + private void processKey(KeyCacheObject key, FilteringVisitorClosure clo) throws IgniteCheckedException { while (true) { try { checkCancelled(); @@ -145,7 +145,7 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor { GridCacheEntryEx entry = cctx.cache().entryEx(key); try { - entry.updateIndex(clo, link); + entry.updateIndex(clo); } finally { cctx.evicts().touch(entry, AffinityTopologyVersion.NONE); @@ -190,15 +190,14 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor { * * @param target Target. */ - public FilteringVisitorClosure(SchemaIndexCacheVisitorClosure target) { + FilteringVisitorClosure(SchemaIndexCacheVisitorClosure target) { this.target = target; } /** {@inheritDoc} */ - @Override public void apply(KeyCacheObject key, int part, CacheObject val, GridCacheVersion ver, - long expiration, long link) throws IgniteCheckedException { - if (qryProc.belongsToTable(cctx, cacheName, tblName, key, val)) - target.apply(key, part, val, ver, expiration, link); + @Override public void apply(CacheDataRow row) throws IgniteCheckedException { + if (qryProc.belongsToTable(cctx, cacheName, tblName, row.key(), row.value())) + target.apply(row); } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 6712b5b..2ba8fd8 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 @@ -842,7 +842,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr } /** {@inheritDoc} */ - @Override public void updateIndex(SchemaIndexCacheVisitorClosure clo, long link) throws IgniteCheckedException, + @Override public void updateIndex(SchemaIndexCacheVisitorClosure clo) throws IgniteCheckedException, GridCacheEntryRemovedException { // No-op. } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 83dd9c9..b0b758a 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 @@ -43,7 +43,7 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +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; @@ -310,15 +310,13 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT } /** {@inheritDoc} */ - @Override public void store(String cacheName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, - CacheObject val, GridCacheVersion ver, long expirationTime, long link) throws IgniteCheckedException { - // No-op + @Override public void store(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow val) { + // No-op. } /** {@inheritDoc} */ - @Override public void remove(String spaceName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, - CacheObject val, GridCacheVersion ver) throws IgniteCheckedException { - // No-op + @Override public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow val) { + // No-op. } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index 9e55442..c3d48dd 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java @@ -361,7 +361,7 @@ public class DmlStatementsProcessor { private UpdateResult executeUpdateStatement(String schemaName, final GridCacheContext cctx, Connection c, Prepared prepared, SqlFieldsQuery fieldsQry, boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel, Object[] failedKeys) throws IgniteCheckedException { - int mainCacheId = CU.cacheId(cctx.name()); + int mainCacheId = cctx.cacheId(); Integer errKeysPos = null; http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 541b80f..ff6ff4d 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 @@ -61,7 +61,6 @@ import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheEntryImpl; -import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheObjectUtils; import org.apache.ignite.internal.processors.cache.CacheObjectValueContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; @@ -72,6 +71,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; import org.apache.ignite.internal.processors.cache.query.CacheQueryPartitionInfo; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable; @@ -79,7 +79,6 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.cache.query.QueryTable; import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.CacheQueryObjectValueContext; import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator; import org.apache.ignite.internal.processors.query.GridQueryCancel; @@ -538,46 +537,47 @@ public class IgniteH2Indexing implements GridQueryIndexing { } /** {@inheritDoc} */ - @Override public void store(String cacheName, - GridQueryTypeDescriptor type, - KeyCacheObject k, - int partId, - CacheObject v, - GridCacheVersion ver, - long expirationTime, - long link) throws IgniteCheckedException { + @Override public void store(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row) + throws IgniteCheckedException { + String cacheName = cctx.name(); + H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name()); if (tbl == null) return; // Type was rejected. - if (expirationTime == 0) - expirationTime = Long.MAX_VALUE; + tbl.table().update(row, false); - tbl.table().update(k, partId, v, ver, expirationTime, false, link); + if (tbl.luceneIndex() != null) { + long expireTime = row.expireTime(); - if (tbl.luceneIndex() != null) - tbl.luceneIndex().store(k, v, ver, expirationTime); + if (expireTime == 0L) + expireTime = Long.MAX_VALUE; + + tbl.luceneIndex().store(row.key(), row.value(), row.version(), expireTime); + } } /** {@inheritDoc} */ - @Override public void remove(String cacheName, - GridQueryTypeDescriptor type, - KeyCacheObject key, - int partId, - CacheObject val, - GridCacheVersion ver) throws IgniteCheckedException { - if (log.isDebugEnabled()) - log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']'); + @Override public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row) + throws IgniteCheckedException + { + if (log.isDebugEnabled()) { + log.debug("Removing key from cache query index [locId=" + nodeId + + ", key=" + row.key() + + ", val=" + row.value() + ']'); + } + + String cacheName = cctx.name(); H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name()); if (tbl == null) return; - if (tbl.table().update(key, partId, val, ver, 0, true, 0)) { + if (tbl.table().update(row, true)) { if (tbl.luceneIndex() != null) - tbl.luceneIndex().remove(key); + tbl.luceneIndex().remove(row.key()); } } @@ -670,14 +670,10 @@ public class IgniteH2Indexing implements GridQueryIndexing { final GridH2RowDescriptor rowDesc = h2Tbl.rowDescriptor(); SchemaIndexCacheVisitorClosure clo = new SchemaIndexCacheVisitorClosure() { - @Override public void apply(KeyCacheObject key, int part, CacheObject val, GridCacheVersion ver, - long expTime, long link) throws IgniteCheckedException { - if (expTime == 0L) - expTime = Long.MAX_VALUE; - - GridH2Row row = rowDesc.createRow(key, part, val, ver, expTime, link); + @Override public void apply(CacheDataRow row) throws IgniteCheckedException { + GridH2Row h2Row = rowDesc.createRow(row); - h2Idx.put(row); + h2Idx.put(h2Row); } }; @@ -2509,7 +2505,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { for (QueryTable tblKey : twoStepQry.tables()) { GridH2Table tbl = dataTable(tblKey); - int cacheId = CU.cacheId(tbl.cacheName()); + int cacheId = tbl.cacheId(); caches0.add(cacheId); } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java index 891e59f..59bf153 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java @@ -31,8 +31,8 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table; import org.apache.ignite.internal.util.lang.GridCursor; -import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter; +import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.h2.engine.Session; import org.h2.index.Cursor; import org.h2.index.IndexType; @@ -197,8 +197,7 @@ public class H2PkHashIndex extends GridH2IndexBase { try { CacheDataRow dataRow = cursor.get(); - return tbl.rowDescriptor().createRow(dataRow.key(), dataRow.partition(), dataRow.value(), - dataRow.version(), 0, dataRow.link()); + return tbl.rowDescriptor().createRow(dataRow); } catch (IgniteCheckedException e) { throw DbException.convert(e); http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java index 7116fe7..40b9b0a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2RowFactory.java @@ -19,11 +19,10 @@ package org.apache.ignite.internal.processors.query.h2.database; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; -import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor; import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor; /** * Data store for H2 rows. @@ -64,8 +63,7 @@ public class H2RowFactory { GridH2Row row; try { - row = rowDesc.createRow(rowBuilder.key(), - PageIdUtils.partId(link), rowBuilder.value(), rowBuilder.version(), rowBuilder.expireTime(), link); + row = rowDesc.createRow(rowBuilder); } catch (IgniteCheckedException e) { throw new IgniteException(e); http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/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 6214be4..fcfeb16 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 @@ -126,7 +126,7 @@ public abstract class H2Tree extends BPlusTree { /** * @return Inline size. */ - public int inlineSize() { + private int inlineSize() { return inlineSize; } @@ -204,6 +204,7 @@ public abstract class H2Tree extends BPlusTree { int idx0 = col.column.getColumnId(); Value v2 = row.getValue(idx0); + if (v2 == null) { // Can't compare further. return 0; @@ -212,6 +213,7 @@ public abstract class H2Tree extends BPlusTree { Value v1 = rowData.getValue(idx0); int c = compareValues(v1, v2); + if (c != 0) return InlineIndexHelper.fixSort(c, col.sortType); } @@ -233,19 +235,28 @@ public abstract class H2Tree extends BPlusTree { for (int i = 0, len = cols.length; i < len; i++) { int idx = columnIds[i]; + Value v1 = r1.getValue(idx); Value v2 = r2.getValue(idx); + if (v1 == null || v2 == null) { - // can't compare further + // Can't compare further. return 0; } + int c = compareValues(v1, v2); + if (c != 0) return InlineIndexHelper.fixSort(c, cols[i].sortType); } + return 0; } - /** Compares two Values. */ + /** + * @param v1 First value. + * @param v2 Second value. + * @return Comparison result. + */ public abstract int compareValues(Value v1, Value v2); } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java index a0716c9..291f8c8 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyRowOnheap.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.query.h2.opt; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.h2.value.Value; /** @@ -27,9 +28,12 @@ public class GridH2KeyRowOnheap extends GridH2Row { private Value key; /** + * @param row Row. * @param key Key. */ - public GridH2KeyRowOnheap(Value key) { + public GridH2KeyRowOnheap(CacheDataRow row, Value key) { + super(row); + this.key = key; } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java index ad93fec..e855536 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2KeyValueRowOnheap.java @@ -18,15 +18,12 @@ package org.apache.ignite.internal.processors.query.h2.opt; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.util.typedef.internal.SB; import org.h2.message.DbException; -import org.h2.result.Row; -import org.h2.result.SearchRow; import org.h2.value.Value; import org.h2.value.ValueNull; -import org.jetbrains.annotations.Nullable; /** * Table row implementation based on {@link GridQueryTypeDescriptor}. @@ -48,10 +45,6 @@ public class GridH2KeyValueRowOnheap extends GridH2Row { protected final GridH2RowDescriptor desc; /** */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - protected long expirationTime; - - /** */ private Value key; /** */ @@ -67,30 +60,24 @@ public class GridH2KeyValueRowOnheap extends GridH2Row { * Constructor. * * @param desc Row descriptor. - * @param key Key. + * @param row Row. * @param keyType Key type. - * @param val Value. * @param valType Value type. - * @param expirationTime Expiration time. * @throws IgniteCheckedException If failed. */ - public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc, Object key, int keyType, @Nullable Object val, - int valType, GridCacheVersion ver, long expirationTime) throws IgniteCheckedException { - this.desc = desc; - this.expirationTime = expirationTime; + public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc, CacheDataRow row, int keyType, int valType) + throws IgniteCheckedException { + super(row); - this.key = desc.wrap(key, keyType); + this.desc = desc; - if (val != null) - this.val = desc.wrap(val, valType); + this.key = desc.wrap(row.key(), keyType); - if (ver != null) - this.ver = desc.wrap(ver, Value.JAVA_OBJECT); - } + if (row.value() != null) + this.val = desc.wrap(row.value(), valType); - /** {@inheritDoc} */ - @Override public long expireTime() { - return expirationTime; + if (row.version() != null) + this.ver = desc.wrap(row.version(), Value.JAVA_OBJECT); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java index 4cb603b..8b1b711 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java @@ -26,75 +26,54 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; * Row with locking support needed for unique key conflicts resolution. */ public abstract class GridH2Row extends GridH2SearchRowAdapter implements CacheDataRow { - /** Link. */ - private long link; + /** Row. */ + private CacheDataRow row; - /** Key. */ - private KeyCacheObject key; - - /** Value. */ - private CacheObject val; - - /** Version. */ - private GridCacheVersion ver; - - /** Partition. */ - private int partId; + /** + * @param row Row. + */ + GridH2Row(CacheDataRow row) { + this.row = row; + } /** {@inheritDoc} */ @Override public KeyCacheObject key() { - return key; + return row.key(); } /** {@inheritDoc} */ @Override public void key(KeyCacheObject key) { - this.key = key; + row.key(key); } /** {@inheritDoc} */ @Override public CacheObject value() { - return val; - } - - /** - * @param val Value. - */ - public void value(CacheObject val) { - this.val = val; + return row.value(); } /** {@inheritDoc} */ @Override public GridCacheVersion version() { - return ver; - } - - /** - * @param ver Version. - */ - public void version(GridCacheVersion ver) { - this.ver = ver; + return row.version(); } /** {@inheritDoc} */ @Override public int partition() { - return partId; + return row.partition(); } - /** - * @param partId Partition. - */ - public void partition(int partId) { - this.partId = partId; + /** {@inheritDoc} */ + @Override public long expireTime() { + return row.expireTime(); } /** {@inheritDoc} */ @Override public long link() { - return link; + return row.link(); } /** {@inheritDoc} */ @Override public void link(long link) { - this.link = link; + row.link(link); } /** {@inheritDoc} */ @@ -104,6 +83,6 @@ public abstract class GridH2Row extends GridH2SearchRowAdapter implements CacheD /** {@inheritDoc} */ @Override public int cacheId() { - return 0; + return row.cacheId(); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java index 503e487..1d915e5 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowDescriptor.java @@ -29,8 +29,7 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.query.GridQueryProperty; import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.h2.H2TableDescriptor; @@ -59,7 +58,6 @@ import org.h2.value.ValueString; import org.h2.value.ValueTime; import org.h2.value.ValueTimestamp; import org.h2.value.ValueUuid; -import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT; import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL; @@ -274,22 +272,18 @@ public class GridH2RowDescriptor { /** * Creates new row. * - * @param key Key. - * @param val Value. - * @param ver Version. - * @param expirationTime Expiration time in millis. + * @param dataRow Data row. * @return Row. * @throws IgniteCheckedException If failed. */ - public GridH2Row createRow(KeyCacheObject key, int partId, @Nullable CacheObject val, GridCacheVersion ver, - long expirationTime, long link) throws IgniteCheckedException { + public GridH2Row createRow(CacheDataRow dataRow) throws IgniteCheckedException { GridH2Row row; try { - if (val == null) // Only can happen for remove operation, can create simple search row. - row = new GridH2KeyRowOnheap(wrap(key, keyType)); + if (dataRow.value() == null) // Only can happen for remove operation, can create simple search row. + row = new GridH2KeyRowOnheap(dataRow, wrap(dataRow.key(), keyType)); else - row = new GridH2KeyValueRowOnheap(this, key, keyType, val, valType, ver, expirationTime); + row = new GridH2KeyValueRowOnheap(this, dataRow, keyType, valType); } catch (ClassCastException e) { throw new IgniteCheckedException("Failed to convert key to SQL type. " + @@ -297,12 +291,6 @@ public class GridH2RowDescriptor { "or configure key type as common super class for all actual keys for this value type.", e); } - row.version(ver); - row.key(key); - row.value(val); - row.partition(partId); - row.link(link); - return row; } http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index add2488..93da34e 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -27,11 +27,9 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteInterruptedException; -import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.query.QueryTable; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory; @@ -222,6 +220,13 @@ public class GridH2Table extends TableBase { } /** + * @return Cache ID. + */ + public int cacheId() { + return cctx.cacheId(); + } + + /** * @return Cache context. */ public GridCacheContext cache() { @@ -389,37 +394,29 @@ public class GridH2Table extends TableBase { * Updates table for given key. If value is null then row with given key will be removed from table, * otherwise value and expiration time will be updated or new row will be added. * - * @param key Key. - * @param val Value. - * @param expirationTime Expiration time. + * @param row Row. * @param rmv If {@code true} then remove, else update row. * @return {@code true} If operation succeeded. * @throws IgniteCheckedException If failed. */ - public boolean update(KeyCacheObject key, - int partId, - CacheObject val, - GridCacheVersion ver, - long expirationTime, - boolean rmv, - long link) + public boolean update(CacheDataRow row, boolean rmv) throws IgniteCheckedException { assert desc != null; - GridH2Row row = desc.createRow(key, partId, val, ver, expirationTime, link); + GridH2Row h2Row = desc.createRow(row); if (rmv) - return doUpdate(row, true); + return doUpdate(h2Row, true); else { - GridH2KeyValueRowOnheap row0 = (GridH2KeyValueRowOnheap)row; + GridH2KeyValueRowOnheap h2Row0 = (GridH2KeyValueRowOnheap)h2Row; - row0.prepareValuesCache(); + h2Row0.prepareValuesCache(); try { - return doUpdate(row, false); + return doUpdate(h2Row0, false); } finally { - row0.clearValuesCache(); + h2Row0.clearValuesCache(); } } } @@ -452,7 +449,7 @@ public class GridH2Table extends TableBase { * @throws IgniteCheckedException If failed. */ @SuppressWarnings("LockAcquiredButNotSafelyReleased") - boolean doUpdate(final GridH2Row row, boolean del) throws IgniteCheckedException { + private boolean doUpdate(final GridH2Row row, boolean del) throws IgniteCheckedException { // Here we assume that each key can't be updated concurrently and case when different indexes // getting updated from different threads with different rows with the same key is impossible. lock(false); @@ -477,15 +474,16 @@ public class GridH2Table extends TableBase { // Put row if absent to all indexes sequentially. // Start from 3 because 0 - Scan (don't need to update), 1 - PK hash (already updated), 2 - PK (already updated). while (++i < len) { - if (!(idxs.get(i) instanceof GridH2IndexBase)) - continue; - GridH2IndexBase idx = index(i); + Index idx = idxs.get(i); - addToIndex(idx, pk, row, old, false); + if (idx instanceof GridH2IndexBase) + addToIndex((GridH2IndexBase)idx, pk, row, old, false); } - for (GridH2IndexBase idx : tmpIdxs.values()) - addToIndex(idx, pk, row, old, true); + if (!tmpIdxs.isEmpty()) { + for (GridH2IndexBase idx : tmpIdxs.values()) + addToIndex(idx, pk, row, old, true); + } } else { // index(1) is PK, get full row from there (search row here contains only key but no other columns). @@ -495,15 +493,19 @@ public class GridH2Table extends TableBase { // Remove row from all indexes. // Start from 3 because 0 - Scan (don't need to update), 1 - PK hash (already updated), 2 - PK (already updated). for (int i = pkIndexPos + 1, len = idxs.size(); i < len; i++) { - if (!(idxs.get(i) instanceof GridH2IndexBase)) - continue; - Row res = index(i).remove(old); + Index idx = idxs.get(i); + + if (idx instanceof GridH2IndexBase) { + Row res = ((GridH2IndexBase)idx).remove(old); - assert eq(pk, res, old) : "\n" + old + "\n" + res + "\n" + i + " -> " + index(i).getName(); + assert eq(pk, res, old) : "\n" + old + "\n" + res + "\n" + i + " -> " + index(i).getName(); + } } - for (GridH2IndexBase idx : tmpIdxs.values()) - idx.remove(old); + if (!tmpIdxs.isEmpty()) { + for (GridH2IndexBase idx : tmpIdxs.values()) + idx.remove(old); + } size.decrement(); } @@ -556,22 +558,6 @@ public class GridH2Table extends TableBase { } /** - * For testing only. - * - * @return Indexes. - */ - ArrayList indexes() { - ArrayList res = new ArrayList<>(idxs.size() - 2); - - for (int i = pkIndexPos, len = idxs.size(); i < len; i++) { - if (idxs.get(i) instanceof GridH2IndexBase) - res.add(index(i)); - } - - return res; - } - - /** * */ public void markRebuildFromHashInProgress(boolean value) { @@ -875,7 +861,7 @@ public class GridH2Table extends TableBase { * @param target Index to clone. * @return Proxy index. */ - public Index createDuplicateIndexIfNeeded(Index target) { + private Index createDuplicateIndexIfNeeded(Index target) { if (!(target instanceof H2TreeIndex) && !(target instanceof SpatialIndex)) return null; http://git-wip-us.apache.org/repos/asf/ignite/blob/879bf581/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java index e58f983..97ef8e5 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java @@ -69,13 +69,9 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.CacheQueryExecutedEvent; import org.apache.ignite.events.CacheQueryReadEvent; import org.apache.ignite.events.Event; -import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.IgniteKernal; -import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.processors.cache.query.QueryCursorEx; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; -import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.lang.GridPlainCallable; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; @@ -1069,16 +1065,15 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac Collection> res = qry.getAll(); - assert res != null; - - assert res.size() == 2; + assertNotNull(res); + assertEquals(2, res.size()); qry = cache.query(new TextQuery(ObjectValue.class, "full")); res = qry.getAll(); - assert res != null; - assert res.size() == 2; + assertNotNull(res); + assertEquals(2, res.size()); } /**