You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2019/01/29 15:26:52 UTC

[ignite] 02/05: Works somehow.

This is an automated email from the ASF dual-hosted git repository.

vozerov pushed a commit to branch ignite-10985
in repository https://gitbox.apache.org/repos/asf/ignite.git

commit d92c02c4ed9a0d3913aac569f8346cba456344de
Author: devozerov <vo...@gridgain.com>
AuthorDate: Tue Jan 29 17:55:12 2019 +0300

    Works somehow.
---
 .../cache/persistence/CacheDataRowAdapter.java     | 128 ++++++++++++++++-----
 .../evict/PageAbstractEvictionTracker.java         |   2 +-
 .../cache/persistence/tree/io/CacheVersionIO.java  |  27 +++++
 .../processors/cache/tree/CacheDataTree.java       |   4 +-
 .../internal/processors/cache/tree/DataRow.java    |   3 +-
 .../processors/query/h2/database/H2RowFactory.java |   3 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java      |   6 +-
 7 files changed, 135 insertions(+), 38 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
index 1889a99..b87d287 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
@@ -75,6 +75,9 @@ public class CacheDataRowAdapter implements CacheDataRow {
     @GridToStringInclude
     protected GridCacheVersion ver;
 
+    /** Whether version is ready. */
+    protected boolean verReady;
+
     /** */
     @GridToStringInclude
     protected int cacheId;
@@ -98,6 +101,8 @@ public class CacheDataRowAdapter implements CacheDataRow {
         this.val = val;
         this.ver = ver;
         this.expireTime = expireTime;
+
+        verReady = true;
     }
 
     /**
@@ -108,7 +113,21 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @throws IgniteCheckedException If failed.
      */
     public final void initFromLink(CacheGroupContext grp, RowData rowData) throws IgniteCheckedException {
-        initFromLink(grp, grp.shared(), grp.dataRegion().pageMemory(), rowData);
+        initFromLink(grp, rowData, false);
+    }
+
+
+    /**
+     * Read row from data pages.
+     *
+     * @param grp Cache group.
+     * @param rowData Required row data.
+     * @param skipVer Whether version read should be skipped.
+     * @throws IgniteCheckedException If failed.
+     */
+    public final void initFromLink(CacheGroupContext grp, RowData rowData, boolean skipVer)
+        throws IgniteCheckedException {
+        initFromLink(grp, grp.shared(), grp.dataRegion().pageMemory(), rowData, skipVer);
     }
 
     /**
@@ -119,13 +138,15 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param sharedCtx Shared context.
      * @param pageMem Page memory.
      * @param rowData Row data.
+     * @param skipVer Whether version read should be skipped.
      * @throws IgniteCheckedException If failed.
      */
     public final void initFromLink(
         @Nullable CacheGroupContext grp,
         GridCacheSharedContext<?, ?> sharedCtx,
         PageMemory pageMem,
-        RowData rowData
+        RowData rowData,
+        boolean skipVer
     ) throws IgniteCheckedException {
         // Group is null if try evict page, with persistence evictions should be disabled.
         assert grp != null || pageMem instanceof PageMemoryNoStoreImpl;
@@ -135,7 +156,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         int grpId = grp != null ? grp.groupId() : 0;
         IoStatisticsHolder statHolder = grp != null ? grp.statisticsHolderData() : IoStatisticsHolderNoOp.INSTANCE;
 
-        doInitFromLink(link, sharedCtx, coctx, pageMem, grpId, statHolder, readCacheId, rowData, null);
+        doInitFromLink(link, sharedCtx, coctx, pageMem, grpId, statHolder, readCacheId, rowData, null, skipVer);
     }
 
     /**
@@ -146,6 +167,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param sharedCtx Cache shared context.
      * @param pageMem Page memory.
      * @param rowData Required row data.
+     * @param skipVer Whether version read should be skipped.
      * @throws IgniteCheckedException If failed.
      */
     public final void initFromDataPage(
@@ -155,7 +177,8 @@ public class CacheDataRowAdapter implements CacheDataRow {
         @Nullable CacheGroupContext grp,
         GridCacheSharedContext<?, ?> sharedCtx,
         PageMemory pageMem,
-        RowData rowData
+        RowData rowData,
+        boolean skipVer
     ) throws IgniteCheckedException {
         // Group is null if try evict page, with persistence evictions should be disabled.
         assert grp != null || pageMem instanceof PageMemoryNoStoreImpl;
@@ -166,14 +189,26 @@ public class CacheDataRowAdapter implements CacheDataRow {
         IoStatisticsHolder statHolder = grp != null ? grp.statisticsHolderData() : IoStatisticsHolderNoOp.INSTANCE;
 
         IncompleteObject<?> incomplete = readIncomplete(null, sharedCtx, coctx, pageMem,
-            grpId, pageAddr, itemId, io, rowData, readCacheId);
+            grpId, pageAddr, itemId, io, rowData, readCacheId, skipVer);
 
         if (incomplete != null) {
             // Initialize the remaining part of the large row from other pages.
             long nextLink = incomplete.getNextLink();
 
-            if (nextLink != 0L)
-                doInitFromLink(nextLink, sharedCtx, coctx, pageMem, grpId, statHolder, readCacheId, rowData, incomplete);
+            if (nextLink != 0L) {
+                doInitFromLink(
+                    nextLink,
+                    sharedCtx,
+                    coctx,
+                    pageMem,
+                    grpId,
+                    statHolder,
+                    readCacheId,
+                    rowData,
+                    incomplete,
+                    skipVer
+                );
+            }
         }
     }
 
@@ -186,6 +221,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param readCacheId {@code true} If need to read cache ID.
      * @param rowData Required row data.
      * @param incomplete Incomplete object.
+     * @param skipVer Whether version read should be skipped.
      * @throws IgniteCheckedException If failed.
      */
     private void doInitFromLink(
@@ -197,7 +233,8 @@ public class CacheDataRowAdapter implements CacheDataRow {
         IoStatisticsHolder statHolder,
         boolean readCacheId,
         RowData rowData,
-        IncompleteObject<?> incomplete
+        IncompleteObject<?> incomplete,
+        boolean skipVer
     ) throws IgniteCheckedException {
         assert link != 0 : "link";
         assert key == null : "key";
@@ -220,7 +257,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
                     int itemId = itemId(nextLink);
 
                     incomplete = readIncomplete(incomplete, sharedCtx, coctx, pageMem,
-                        grpId, pageAddr, itemId, io, rowData, readCacheId);
+                        grpId, pageAddr, itemId, io, rowData, readCacheId, skipVer);
 
                     if (incomplete == null || (rowData == KEY_ONLY && key != null))
                         return;
@@ -250,6 +287,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param io Page IO.
      * @param rowData Required row data.
      * @param readCacheId {@code true} If need to read cache ID.
+     * @param skipVer Whether version read should be skipped.
      * @return Incomplete object.
      * @throws IgniteCheckedException If failed.
      */
@@ -263,7 +301,8 @@ public class CacheDataRowAdapter implements CacheDataRow {
         int itemId,
         DataPageIO io,
         RowData rowData,
-        boolean readCacheId
+        boolean readCacheId,
+        boolean skipVer
     ) throws IgniteCheckedException {
         DataPagePayload data = io.readPayload(pageAddr, itemId, pageMem.realPageSize(grpId));
 
@@ -274,7 +313,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         if (incomplete == null) {
             if (nextLink == 0) {
                 // Fast path for a single page row.
-                readFullRow(sharedCtx, coctx, pageAddr + data.offset(), rowData, readCacheId);
+                readFullRow(sharedCtx, coctx, pageAddr + data.offset(), rowData, readCacheId, skipVer);
 
                 return null;
             }
@@ -296,7 +335,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
         boolean keyOnly = rowData == RowData.KEY_ONLY;
 
-        incomplete = readFragment(sharedCtx, coctx, buf, keyOnly, readCacheId, incomplete);
+        incomplete = readFragment(sharedCtx, coctx, buf, keyOnly, readCacheId, incomplete, skipVer);
 
         if (incomplete != null)
             incomplete.setNextLink(nextLink);
@@ -323,6 +362,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param keyOnly {@code true} If need to read only key object.
      * @param readCacheId {@code true} If need to read cache ID.
      * @param incomplete Incomplete object.
+     * @param skipVer Whether version read should be skipped.
      * @throws IgniteCheckedException If failed.
      * @return Read object.
      */
@@ -332,7 +372,8 @@ public class CacheDataRowAdapter implements CacheDataRow {
         ByteBuffer buf,
         boolean keyOnly,
         boolean readCacheId,
-        IncompleteObject<?> incomplete
+        IncompleteObject<?> incomplete,
+        boolean skipVer
     ) throws IgniteCheckedException {
         if (readCacheId && cacheId == 0) {
             incomplete = readIncompleteCacheId(buf, incomplete);
@@ -392,10 +433,10 @@ public class CacheDataRowAdapter implements CacheDataRow {
         }
 
         // Read version.
-        if (ver == null) {
-            incomplete = readIncompleteVersion(buf, incomplete);
+        if (!verReady) {
+            incomplete = readIncompleteVersion(buf, incomplete, skipVer);
 
-            assert ver != null || incomplete != null;
+            assert skipVer || ver != null || incomplete != null;
         }
 
         return incomplete;
@@ -407,6 +448,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param addr Address.
      * @param rowData Required row data.
      * @param readCacheId {@code true} If need to read cache ID.
+     * @param skipVer Whether version read should be skipped.
      * @throws IgniteCheckedException If failed.
      */
     protected void readFullRow(
@@ -414,8 +456,9 @@ public class CacheDataRowAdapter implements CacheDataRow {
         CacheObjectContext coctx,
         long addr,
         RowData rowData,
-        boolean readCacheId)
-        throws IgniteCheckedException {
+        boolean readCacheId,
+        boolean skipVer
+    ) throws IgniteCheckedException {
         int off = 0;
 
         off += readHeader(addr, off);
@@ -461,9 +504,22 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
         val = coctx.kernalContext().cacheObjects().toCacheObject(coctx, type, bytes);
 
-        ver = CacheVersionIO.read(addr + off, false);
+        int verLen;
+
+        if (skipVer) {
+            ver = null;
+
+            verLen = CacheVersionIO.readSize(addr + off, false);
+        }
+        else {
+            ver = CacheVersionIO.read(addr + off, false);
+
+            verLen = CacheVersionIO.size(ver, false);
+        }
 
-        off += CacheVersionIO.size(ver, false);
+        verReady = true;
+
+        off += verLen;
 
         expireTime = PageUtils.getLong(addr, off);
     }
@@ -604,12 +660,14 @@ public class CacheDataRowAdapter implements CacheDataRow {
     /**
      * @param buf Buffer.
      * @param incomplete Incomplete object.
+     * @param skip Whether version should be skipped.
      * @return Incomplete object.
      * @throws IgniteCheckedException If failed.
      */
     protected IncompleteObject<?> readIncompleteVersion(
         ByteBuffer buf,
-        IncompleteObject<?> incomplete
+        IncompleteObject<?> incomplete,
+        boolean skip
     ) throws IgniteCheckedException {
         if (incomplete == null || incomplete.data() == null) {
             int remaining = buf.remaining();
@@ -621,10 +679,16 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
             if (remaining >= size) {
                 // If the whole version is on a single page, just read it.
-                ver = CacheVersionIO.read(buf, false);
+                if (skip)
+                    buf.position(buf.position() + size);
+                else {
+                    ver = CacheVersionIO.read(buf, false);
 
-                assert !buf.hasRemaining(): buf.remaining();
-                assert ver != null;
+                    assert !buf.hasRemaining(): buf.remaining();
+                    assert ver != null;
+                }
+
+                verReady = true;
 
                 return null;
             }
@@ -636,13 +700,17 @@ public class CacheDataRowAdapter implements CacheDataRow {
         incomplete.readData(buf);
 
         if (incomplete.isReady()) {
-            final ByteBuffer verBuf = ByteBuffer.wrap(incomplete.data());
+            if (!skip) {
+                ByteBuffer verBuf = ByteBuffer.wrap(incomplete.data());
+
+                verBuf.order(buf.order());
 
-            verBuf.order(buf.order());
+                ver = CacheVersionIO.read(verBuf, false);
 
-            ver = CacheVersionIO.read(verBuf, false);
+                assert ver != null;
+            }
 
-            assert ver != null;
+            verReady = true;
         }
 
         assert !buf.hasRemaining();
@@ -654,7 +722,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @return {@code True} if entry is ready.
      */
     public boolean isReady() {
-        return ver != null && val != null && key != null;
+        return verReady && val != null && key != null;
     }
 
     /** {@inheritDoc} */
@@ -687,7 +755,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
     /** {@inheritDoc} */
     @Override public GridCacheVersion version() {
-        assert ver != null : "Version is not ready: " + this;
+        assert verReady : "Version is not ready: " + this;
 
         return ver;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
index 5142c59..41731b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
@@ -102,7 +102,7 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
                     @Override public CacheDataRowAdapter apply(long link) throws IgniteCheckedException {
                         CacheDataRowAdapter row = new CacheDataRowAdapter(link);
 
-                        row.initFromLink(null, sharedCtx, pageMem, CacheDataRowAdapter.RowData.KEY_ONLY);
+                        row.initFromLink(null, sharedCtx, pageMem, CacheDataRowAdapter.RowData.KEY_ONLY, false);
 
                         assert row.cacheId() != 0 : "Cache ID should be stored in rows of evictable cache";
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CacheVersionIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CacheVersionIO.java
index d9fdf0e..677c3fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CacheVersionIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/CacheVersionIO.java
@@ -171,6 +171,33 @@ public class CacheVersionIO {
     }
 
     /**
+     * Gets needed buffer size to read the whole version instance.
+     * Does not change buffer position.
+     *
+     * @param pageAddr Page address.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Size of serialized version.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static int readSize(long pageAddr, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(PageUtils.getByte(pageAddr, 0), allowNull);
+
+        switch (protoVer) {
+            case NULL_PROTO_VER:
+                return NULL_SIZE;
+
+            case 1:
+                return SIZE_V1;
+
+            case 2:
+                return SIZE_V2;
+
+            default:
+                throw new IllegalStateException();
+        }
+    }
+
+    /**
      * Reads GridCacheVersion instance from the given buffer. Moves buffer's position by the number of used
      * bytes.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
index 4d01494..4349823 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
@@ -246,7 +246,9 @@ public class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
                             for (int i = 0; i < rowsCnt; i++) {
                                 if (c == null || c.applyMvcc(io, pageAddr, i, pageSize)) {
                                     DataRow row = mvccEnabled ? new MvccDataRow() : new DataRow();
-                                    row.initFromDataPage(io, pageAddr, i, grp, shared, pageMem, rowData);
+
+                                    row.initFromDataPage(io, pageAddr, i, grp, shared, pageMem, rowData, true);
+
                                     rows[r++] = row;
                                 }
                             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java
index ca6061b..552a39c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java
@@ -46,7 +46,6 @@ public class DataRow extends CacheDataRowAdapter {
         super(link);
 
         this.hash = hash;
-
         this.part = part;
 
         try {
@@ -80,6 +79,8 @@ public class DataRow extends CacheDataRowAdapter {
         this.part = part;
         this.expireTime = expireTime;
         this.cacheId = cacheId;
+
+        verReady = true;
     }
 
     /**
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 5b4827b..bb17706 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
@@ -58,7 +58,8 @@ public class H2RowFactory {
 
         row.initFromLink(
             cctx.group(),
-            CacheDataRowAdapter.RowData.FULL // TODO: Appropriate data!
+            CacheDataRowAdapter.RowData.FULL,
+            true
         );
 
         return rowDesc.createRow(row);
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 a260b4d..ae6cb33 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
@@ -61,10 +61,8 @@ public class GridH2KeyValueRowOnheap extends GridH2Row {
      * @param valType Value type.
      * @throws IgniteCheckedException If failed.
      */
-    public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc,
-        CacheDataRow row,
-        int keyType,
-        int valType) throws IgniteCheckedException {
+    public GridH2KeyValueRowOnheap(GridH2RowDescriptor desc, CacheDataRow row, int keyType, int valType)
+        throws IgniteCheckedException {
         super(row);
 
         this.desc = desc;