You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2017/02/16 14:40:53 UTC

[01/17] ignite git commit: ignite-4652 - minor

Repository: ignite
Updated Branches:
  refs/heads/ignite-4652 8b68231e0 -> affadf3bb


ignite-4652 - minor


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1fd247f7
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1fd247f7
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1fd247f7

Branch: refs/heads/ignite-4652
Commit: 1fd247f712f00d56132c8259386f28c8430053ba
Parents: 42b7f47
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Feb 6 16:42:28 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Feb 6 16:42:28 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 44 ++++++++++++++------
 1 file changed, 31 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1fd247f7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 78ebb3b..39b8b10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -68,9 +68,9 @@ import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTre
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Bool.FALSE;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Bool.READY;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Bool.TRUE;
-import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.DELETE;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.INSERT;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.NOOP;
+import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.REMOVE;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.REPLACE;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.FOUND;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.GO_DOWN;
@@ -82,7 +82,6 @@ import static org.apache.ignite.internal.processors.cache.database.tree.util.Pag
 import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.isWalDeltaRecordNeeded;
 import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.readPage;
 import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.writePage;
-import static org.apache.ignite.internal.util.IgniteTree.OperationType.REMOVE;
 
 /**
  * Abstract B+Tree.
@@ -1563,13 +1562,30 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     case FOUND:
                         // We must be at the bottom here, just need to remove row from the current page.
                         assert lvl == 0 : lvl;
-                        assert r.invokeType != null;
 
-                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
+                        switch (r.invokeType) {
+                            case REPLACE: {
+                                // TODO replace
+                                break;
+                            }
 
-                        if (res == FOUND && r.tail == null) {
-                            // Finish if we don't need to do any merges.
-                            r.finish();
+                            case REMOVE: {
+                                res = r.removeFromLeaf(pageId, page, backId, fwdId);
+
+                                if (res == FOUND && r.tail == null) {
+                                    // Finish if we don't need to do any merges.
+                                    r.finish();
+                                }
+
+                                break;
+                            }
+
+                            case NOOP:
+                                break;
+
+                            case INSERT: // We can not have INSERT here since we have found a row.
+                            default:
+                                throw new IllegalStateException("Type: " + r.invokeType);
                         }
 
                         return res;
@@ -2506,10 +2522,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @return Operation type or {@code null} if it is unknown yet.
+         * @return Operation type or {@code null} if it is unknown yet. When the closure is invoked
+         *      we must know exact operation type. Note that even if the operation type is known
+         *      from the beginning it is allowed to change after the closure invocation, for example
+         *      initially it was {@code PUT} but became {@code NOOP}.
          */
         private OperationType getOperationType() {
-            return c == null ? REMOVE : c.operationType();
+            return c == null ? OperationType.REMOVE : c.operationType();
         }
 
         /**
@@ -2521,8 +2540,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             OperationType opType = getOperationType();
 
-            if (opType == null)
-                return;
+            assert opType != null; // We do this always after the closure has been invoked.
 
             switch (opType) {
                 case NOOP:
@@ -2534,7 +2552,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     break;
 
                 case REMOVE:
-                    invokeType = DELETE;
+                    invokeType = REMOVE;
                     break;
 
                 default:
@@ -4063,7 +4081,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         REPLACE,
 
         /** */
-        DELETE,
+        REMOVE,
 
         /** */
         NOOP


[02/17] ignite git commit: ignite-4652 - reverted

Posted by se...@apache.org.
ignite-4652 - reverted


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/164995f6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/164995f6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/164995f6

Branch: refs/heads/ignite-4652
Commit: 164995f601c188a45275d2f2d0dbbc0b6a742523
Parents: 1fd247f
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Feb 6 18:23:29 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Feb 6 18:23:29 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 197 ++++---------------
 1 file changed, 37 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/164995f6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 39b8b10..860f542 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -68,10 +68,6 @@ import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTre
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Bool.FALSE;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Bool.READY;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Bool.TRUE;
-import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.INSERT;
-import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.NOOP;
-import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.REMOVE;
-import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.InvokeType.REPLACE;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.FOUND;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.GO_DOWN;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.GO_DOWN_X;
@@ -228,9 +224,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             long res = doAskNeighbor(io, pageAddr, back);
 
             if (back) {
-                assert g.getClass() == Invoke.class;
+                assert g.getClass() == Remove.class;
 
-                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in invokeDown for this check.
+                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
                     return RETRY;
 
                 g.backId = res;
@@ -423,14 +419,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /** */
-    private final GetPageHandler<Invoke> rmvFromLeaf = new RemoveFromLeaf();
+    private final GetPageHandler<Remove> rmvFromLeaf = new RemoveFromLeaf();
 
     /**
      *
      */
-    private class RemoveFromLeaf extends GetPageHandler<Invoke> {
+    private class RemoveFromLeaf extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
+        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
@@ -494,14 +490,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /** */
-    private final GetPageHandler<Invoke> lockBackAndRmvFromLeaf = new LockBackAndRmvFromLeaf();
+    private final GetPageHandler<Remove> lockBackAndRmvFromLeaf = new LockBackAndRmvFromLeaf();
 
     /**
      *
      */
-    private class LockBackAndRmvFromLeaf extends GetPageHandler<Invoke> {
+    private class LockBackAndRmvFromLeaf extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
+        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(pageAddr) != r.pageId)
@@ -519,14 +515,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /** */
-    private final GetPageHandler<Invoke> lockBackAndTail = new LockBackAndTail();
+    private final GetPageHandler<Remove> lockBackAndTail = new LockBackAndTail();
 
     /**
      *
      */
-    private class LockBackAndTail extends GetPageHandler<Invoke> {
+    private class LockBackAndTail extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
+        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(pageAddr) != r.pageId)
@@ -543,14 +539,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /** */
-    private final GetPageHandler<Invoke> lockTailForward = new LockTailForward();
+    private final GetPageHandler<Remove> lockTailForward = new LockTailForward();
 
     /**
      *
      */
-    private class LockTailForward extends GetPageHandler<Invoke> {
+    private class LockTailForward extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
+        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
 
@@ -559,14 +555,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /** */
-    private final GetPageHandler<Invoke> lockTail = new LockTail();
+    private final GetPageHandler<Remove> lockTail = new LockTail();
 
     /**
      *
      */
-    private class LockTail extends GetPageHandler<Invoke> {
+    private class LockTail extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
@@ -1405,42 +1401,27 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         return res != null ? res : false;
     }
 
-    /**
-     * @param row Lookup row.
-     * @param needOld {@code True} if need return removed row.
-     * @return Removed row.
-     * @throws IgniteCheckedException If failed.
-     */
-    private T doRemove(L row, boolean needOld) throws IgniteCheckedException {
-        return doInvoke(row, needOld, null);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
+    /** {@inheritDoc} */
     @Override public void invoke(L key, InvokeClosure<T> c) throws IgniteCheckedException {
-        assert c != null;
-
-        doInvoke(key, false, c);
+        // TODO
     }
 
     /**
      * @param row Lookup row.
      * @param needOld {@code True} if need return removed row.
-     * @param c Invoke closure or {@code null} if it is just a plain remove operation.
      * @return Removed row.
      * @throws IgniteCheckedException If failed.
      */
-    private T doInvoke(L row, boolean needOld, InvokeClosure<T> c) throws IgniteCheckedException {
+    private T doRemove(L row, boolean needOld) throws IgniteCheckedException {
         checkDestroyed();
 
-        Invoke r = new Invoke(row, needOld, c);
+        Remove r = new Remove(row, needOld);
 
         try {
             for (;;) {
                 r.init();
 
-                switch (invokeDown(r, r.rootId, 0L, 0L, r.rootLvl)) {
+                switch (removeDown(r, r.rootId, 0L, 0L, r.rootLvl)) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1465,7 +1446,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         assert r.isFinished();
 
-                        return r.oldRow;
+                        return r.rmvd;
                 }
             }
         }
@@ -1494,7 +1475,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return Result code.
      * @throws IgniteCheckedException If failed.
      */
-    private Result invokeDown(final Invoke r, final long pageId, final long backId, final long fwdId, final int lvl)
+    private Result removeDown(final Remove r, final long pageId, final long backId, final long fwdId, final int lvl)
         throws IgniteCheckedException {
         assert lvl >= 0 : lvl;
 
@@ -1529,7 +1510,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         // Intentional fallthrough.
                     case GO_DOWN:
-                        res = invokeDown(r, r.pageId, r.backId, r.fwdId, lvl - 1);
+                        res = removeDown(r, r.pageId, r.backId, r.fwdId, lvl - 1);
 
                         if (res == RETRY) {
                             checkInterrupted();
@@ -1549,13 +1530,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     case NOT_FOUND:
                         // We are at the bottom.
                         assert lvl == 0 : lvl;
-                        assert r.invokeType != null;
 
-                        if (r.invokeType == INSERT) {
-                            // TODO insert
-                        }
-                        else
-                            r.finish();
+                        r.finish();
 
                         return res;
 
@@ -1563,29 +1539,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         // We must be at the bottom here, just need to remove row from the current page.
                         assert lvl == 0 : lvl;
 
-                        switch (r.invokeType) {
-                            case REPLACE: {
-                                // TODO replace
-                                break;
-                            }
-
-                            case REMOVE: {
-                                res = r.removeFromLeaf(pageId, page, backId, fwdId);
+                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
 
-                                if (res == FOUND && r.tail == null) {
-                                    // Finish if we don't need to do any merges.
-                                    r.finish();
-                                }
-
-                                break;
-                            }
-
-                            case NOOP:
-                                break;
-
-                            case INSERT: // We can not have INSERT here since we have found a row.
-                            default:
-                                throw new IllegalStateException("Type: " + r.invokeType);
+                        if (res == FOUND && r.tail == null) {
+                            // Finish if we don't need to do any merges.
+                            r.finish();
                         }
 
                         return res;
@@ -2478,7 +2436,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * Remove operation.
      */
-    private final class Invoke extends Get implements ReuseBag {
+    private final class Remove extends Get implements ReuseBag {
         /** We may need to lock part of the tree branch from the bottom to up for multiple levels. */
         private Tail<L> tail;
 
@@ -2488,8 +2446,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         Bool needMergeEmptyBranch = FALSE;
 
-        /** Updated or removed row. */
-        private T oldRow;
+        /** Removed row. */
+        private T rmvd;
 
         /** Current page. */
         private Page page;
@@ -2500,64 +2458,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         private final boolean needOld;
 
-        /** */
-        private final InvokeClosure<T> c;
-
-        /** */
-        private boolean closureInvoked;
-
-        /** */
-        private InvokeType invokeType;
-
         /**
          * @param row Row.
          * @param needOld {@code True} If need return old value.
-         * @param c Invoke closure or {@code null} if it is just a plain remove operation.
          */
-        private Invoke(L row, boolean needOld, InvokeClosure<T> c) {
+        private Remove(L row, boolean needOld) {
             super(row);
 
             this.needOld = needOld;
-            this.c = c;
-        }
-
-        /**
-         * @return Operation type or {@code null} if it is unknown yet. When the closure is invoked
-         *      we must know exact operation type. Note that even if the operation type is known
-         *      from the beginning it is allowed to change after the closure invocation, for example
-         *      initially it was {@code PUT} but became {@code NOOP}.
-         */
-        private OperationType getOperationType() {
-            return c == null ? OperationType.REMOVE : c.operationType();
-        }
-
-        /**
-         * @param rowFound If the old row was found.
-         */
-        private void setupInvokeType(boolean rowFound) {
-            if (invokeType != null)
-                return;
-
-            OperationType opType = getOperationType();
-
-            assert opType != null; // We do this always after the closure has been invoked.
-
-            switch (opType) {
-                case NOOP:
-                    invokeType = NOOP;
-                    break;
-
-                case PUT:
-                    invokeType = rowFound ? REPLACE : INSERT;
-                    break;
-
-                case REMOVE:
-                    invokeType = REMOVE;
-                    break;
-
-                default:
-                    throw new IllegalStateException("Operation type: " + opType);
-            }
         }
 
         /** {@inheritDoc} */
@@ -2603,41 +2511,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             if (lvl == 0) {
                 assert tail == null;
 
-                doInvokeClosure(null, 0L, 0);
-
-                return true;
-            }
-
-            return false;
-        }
-
-        /**
-         * @param io Page IO.
-         * @param pageAddr Page address.
-         * @param idx Index of found entry.
-         * @throws IgniteCheckedException If failed.
-         */
-        private void doInvokeClosure(BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException {
-            boolean rowFound = io != null;
-
-            if (c != null && !closureInvoked) {
-                c.call(rowFound ? getRow(io, pageAddr, idx) : null);
-
-                closureInvoked = true;
-            }
-
-            setupInvokeType(rowFound);
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
-            if (lvl == 0) {
-                doInvokeClosure(io, pageAddr, idx);
-
                 return true;
             }
 
@@ -2732,7 +2609,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return {@code true} If already removed from leaf.
          */
         private boolean isRemoved() {
-            return oldRow != null;
+            return rmvd != null;
         }
 
         /**
@@ -3009,7 +2886,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert !isRemoved(): "already removed";
 
             // Detach the row.
-            oldRow = needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
+            rmvd = needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
 
             doRemove(page, io, pageAddr, cnt, idx);
 


[09/17] ignite git commit: ignite-db-x - wip2

Posted by se...@apache.org.
ignite-db-x - wip2


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/94174121
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/94174121
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/94174121

Branch: refs/heads/ignite-4652
Commit: 94174121522f318b2df0564729b9a81cbd1a3a42
Parents: 4f1088f
Author: Sergi Vladykin <se...@gmail.com>
Authored: Tue Feb 14 15:59:18 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Tue Feb 14 15:59:18 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/database/tree/BPlusTree.java         | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94174121/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 51d2c83..c21032f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -358,7 +358,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Inner replace state must be consistent by the end of the operation.
                 assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
 
-                p.finish();
+                // Need to replace inner key if now we are replacing the rightmost row and have a forward page.
+                if (canGetRowFromInner && idx + 1 == cnt && p.fwdId != 0L && p.needReplaceInner == FALSE) {
+                    // Can happen only for invoke, otherwise inner key must be replaced on the way down.
+                    assert p.invoke;
+
+                    p.needReplaceInner = TRUE;
+                }
+                else
+                    p.finish();
             }
 
             io.store(pageAddr, idx, newRow, null);


[13/17] ignite git commit: ignite-4652 - minor

Posted by se...@apache.org.
ignite-4652 - minor


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/607e4187
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/607e4187
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/607e4187

Branch: refs/heads/ignite-4652
Commit: 607e418750b70bdb4bf27cf25a1bd77e7dd7d58b
Parents: b4283a3
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed Feb 15 19:19:52 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Feb 15 19:19:52 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/database/tree/BPlusTree.java     | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/607e4187/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 570569f..904d91e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -1560,7 +1560,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             for (;;) {
                 r.init();
 
-                switch (removeDown(r, r.rootId, 0L, 0L, r.rootLvl)) {
+                Result res = removeDown(r, r.rootId, 0L, 0L, r.rootLvl);
+
+                switch (res) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1569,7 +1571,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                     default:
                         if (!r.isFinished()) {
-                            Result res = r.finishTail();
+                            res = r.finishTail();
 
                             // If not found, then the tree grew beyond our call stack -> retry from the actual root.
                             if (res == RETRY || res == NOT_FOUND) {


[05/17] ignite git commit: ignite-db-x - Minor.

Posted by se...@apache.org.
ignite-db-x - Minor.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7689b364
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7689b364
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7689b364

Branch: refs/heads/ignite-4652
Commit: 7689b36427d1cdcbb19b59fdb3d048a93c417af0
Parents: d8ff795
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Feb 13 12:09:07 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Feb 13 12:09:07 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 109 +++++++++++++------
 1 file changed, 78 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7689b364/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 68a0255..e126923 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -227,12 +227,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
                     return RETRY;
 
-                g.backId = res;
+                g.backId(res);
             }
             else {
                 assert isBack == FALSE.ordinal() : isBack;
 
-                g.fwdId = res;
+                g.fwdId(res);
             }
 
             return FOUND;
@@ -255,7 +255,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             boolean needBackIfRouting = g.backId != 0;
 
-            g.backId = 0; // Usually we'll go left down and don't need it.
+            g.backId(0L); // Usually we'll go left down and don't need it.
 
             int cnt = io.getCount(pageAddr);
             int idx = findInsertionPoint(io, pageAddr, 0, cnt, g.row, g.shift);
@@ -280,13 +280,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert !io.isLeaf() : io;
 
             // If idx == cnt then we go right down, else left down: getLeft(cnt) == getRight(cnt - 1).
-            g.pageId = inner(io).getLeft(pageAddr, idx);
+            g.pageId(inner(io).getLeft(pageAddr, idx));
 
             // If we see the tree in consistent state, then our right down page must be forward for our left down page,
             // we need to setup fwdId and/or backId to be able to check this invariant on lower level.
             if (idx < cnt) {
                 // Go left down here.
-                g.fwdId = inner(io).getRight(pageAddr, idx);
+                g.fwdId(inner(io).getRight(pageAddr, idx));
             }
             else {
                 // Go right down here or it is an empty branch.
@@ -299,7 +299,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // Setup fwdId.
                 if (fwdId == 0)
-                    g.fwdId = 0;
+                    g.fwdId(0L);
                 else {
                     // We can do askNeighbor on forward page here because we always take locks in forward direction.
                     Result res = askNeighbor(fwdId, g, false);
@@ -310,7 +310,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // Setup backId.
                 if (cnt != 0) // It is not a routing page and we are going to the right, can get backId here.
-                    g.backId = inner(io).getLeft(pageAddr, cnt - 1);
+                    g.backId(inner(io).getLeft(pageAddr, cnt - 1));
                 else if (needBackIfRouting) {
                     // Can't get backId here because of possible deadlock and it is only needed for remove operation.
                     return GO_DOWN_X;
@@ -1401,6 +1401,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** {@inheritDoc} */
     @Override public void invoke(L row, InvokeClosure<T> c) throws IgniteCheckedException {
+        assert c != null;
+
         checkDestroyed();
 
         Invoke x = new Invoke(row, c);
@@ -1457,9 +1459,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         try {
             for (;;) {
                 // Init args.
-                x.pageId = pageId;
-                x.fwdId = fwdId;
-                x.backId = backId;
+                x.pageId(pageId);
+                x.fwdId(fwdId);
+                x.backId(backId);
 
                 Result res = readPage(page, this, search, x, lvl, RETRY);
 
@@ -1468,7 +1470,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         assert backId != 0;
                         assert x.backId == 0; // We did not setup it yet.
 
-                        x.backId = pageId; // Dirty hack to setup a check inside of askNeighbor.
+                        x.backId(pageId); // Dirty hack to setup a check inside of askNeighbor.
 
                         // We need to get backId here for our child page, it must be the last child of our back.
                         res = askNeighbor(backId, x, true);
@@ -1493,9 +1495,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                                 case DONE:
                                     return res;
 
-                                case
-
-
+                                    // TODO
                             }
                         }
 
@@ -1503,22 +1503,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                     case NOT_FOUND:
                         // We are at the bottom.
-                        assert lvl == 0 : lvl;
-
-                        r.finish();
+//                        assert lvl == 0 : lvl;
+//
+//                        r.finish();
 
                         return res;
 
                     case FOUND:
                         // We must be at the bottom here, just need to remove row from the current page.
-                        assert lvl == 0 : lvl;
-
-                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
-
-                        if (res == FOUND && r.tail == null) {
-                            // Finish if we don't need to do any merges.
-                            r.finish();
-                        }
+//                        assert lvl == 0 : lvl;
+//
+//                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
+//
+//                        if (res == FOUND && r.tail == null) {
+//                            // Finish if we don't need to do any merges.
+//                            r.finish();
+//                        }
 
                         return res;
 
@@ -2188,15 +2188,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param g Other operation to copy from.
+         * @return {@code this}.
          */
-        final void copyFrom(Get g) {
+        final Get copyFrom(Get g) {
             rmvId = g.rmvId;
             rootLvl = g.rootLvl;
-            row = g.row;
             pageId = g.pageId;
             fwdId = g.fwdId;
             backId = g.backId;
             shift = g.shift;
+
+            return this;
         }
 
         /**
@@ -2259,6 +2261,27 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         boolean canRelease(Page page, int lvl) {
             return page != null;
         }
+
+        /**
+         * @param backId Back page ID.
+         */
+        void backId(long backId) {
+            this.backId = backId;
+        }
+
+        /**
+         * @param pageId Page ID.
+         */
+        void pageId(long pageId) {
+            this.pageId = pageId;
+        }
+
+        /**
+         * @param fwdId Forward page ID.
+         */
+        void fwdId(long fwdId) {
+            this.fwdId = fwdId;
+        }
     }
 
     /**
@@ -2598,6 +2621,30 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
+        @Override void pageId(long pageId) {
+            this.pageId = pageId;
+
+            if (op != null)
+                op.pageId = pageId;
+        }
+
+        /** {@inheritDoc} */
+        @Override void fwdId(long fwdId) {
+            this.fwdId = fwdId;
+
+            if (op != null)
+                op.fwdId = fwdId;
+        }
+
+        /** {@inheritDoc} */
+        @Override void backId(long backId) {
+            this.backId = backId;
+
+            if (op != null)
+                op.backId = backId;
+        }
+
+        /** {@inheritDoc} */
         @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             if (lvl == 0) {
                 invokeClosure(io, pageAddr, idx);
@@ -2629,24 +2676,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (closureInvoked)
                 return;
 
+            closureInvoked = true;
+
             boolean rowFound = io != null;
 
             clo.call(rowFound ? getRow(io, pageAddr, idx) : null);
 
-            closureInvoked = true;
-
             switch (clo.operationType()) {
                 case PUT:
                     T newRow = clo.newRow();
 
                     assert newRow != null;
 
-                    op = new Put(newRow, false);
+                    op = new Put(newRow, false).copyFrom(this);
 
                     break;
 
                 case REMOVE:
-                    op = new Remove(row, false);
+                    op = new Remove(row, false).copyFrom(this);
 
                     break;
 


[07/17] ignite git commit: ignite-db-x - restartFromRoot

Posted by se...@apache.org.
ignite-db-x - restartFromRoot


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/9fe01a49
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/9fe01a49
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/9fe01a49

Branch: refs/heads/ignite-4652
Commit: 9fe01a494d97f98a5cbf2873046a8bb1a38b69d7
Parents: 97e9bd9
Author: Sergi Vladykin <se...@gmail.com>
Authored: Tue Feb 14 13:20:36 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Tue Feb 14 13:20:36 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/database/tree/BPlusTree.java         | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe01a49/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index f87ba2b..7770c27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -2198,7 +2198,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @param rootLvl Root level.
          * @param rmvId Remove ID to be afraid of.
          */
-        final void restartFromRoot(long rootId, int rootLvl, long rmvId) {
+        void restartFromRoot(long rootId, int rootLvl, long rmvId) {
             this.rootId = rootId;
             this.rootLvl = rootLvl;
             this.rmvId = rmvId;
@@ -2660,6 +2660,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
+        @Override void restartFromRoot(long rootId, int rootLvl, long rmvId) {
+            super.restartFromRoot(rootId, rootLvl, rmvId);
+
+            if (op != null)
+                op.restartFromRoot(rootId, rootLvl, rmvId);
+        }
+
+        /** {@inheritDoc} */
         @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             if (lvl == 0) {
                 invokeClosure(io, pageAddr, idx);


[04/17] ignite git commit: ignite-db-x - Removed code duplications.

Posted by se...@apache.org.
ignite-db-x - Removed code duplications.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d8ff7959
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d8ff7959
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d8ff7959

Branch: refs/heads/ignite-4652
Commit: d8ff795999bfccdb2e8650e7a3a2753153ea2a1e
Parents: d58d0f1
Author: Sergi Vladykin <se...@gmail.com>
Authored: Fri Feb 10 17:28:05 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Fri Feb 10 17:28:05 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 308 +++++++++++++++++--
 1 file changed, 284 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d8ff7959/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 860f542..68a0255 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -224,8 +224,6 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             long res = doAskNeighbor(io, pageAddr, back);
 
             if (back) {
-                assert g.getClass() == Remove.class;
-
                 if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
                     return RETRY;
 
@@ -1402,11 +1400,143 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /** {@inheritDoc} */
-    @Override public void invoke(L key, InvokeClosure<T> c) throws IgniteCheckedException {
-        // TODO
+    @Override public void invoke(L row, InvokeClosure<T> c) throws IgniteCheckedException {
+        checkDestroyed();
+
+        Invoke x = new Invoke(row, c);
+
+        try {
+            for (;;) {
+                x.init();
+
+                switch (invokeDown(x, x.rootId, 0L, 0L, x.rootLvl)) {
+                    case RETRY:
+                    case RETRY_ROOT:
+                        checkInterrupted();
+
+                        continue;
+
+                    default:
+                        if (x.finishIfNeeded() != FALSE)
+                            return;
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Runtime failure on search row: " + row, e);
+        }
+        catch (RuntimeException e) {
+            throw new IgniteException("Runtime failure on search row: " + row, e);
+        }
+        catch (AssertionError e) {
+            throw new AssertionError("Assertion error on search row: " + row, e);
+        }
+        finally {
+            x.releaseAll();
+        }
     }
 
     /**
+     * @param x Invoke operation.
+     * @param pageId Page ID.
+     * @param backId Expected backward page ID if we are going to the right.
+     * @param fwdId Expected forward page ID.
+     * @param lvl Level.
+     * @return Result code.
+     * @throws IgniteCheckedException If failed.
+     */
+    private Result invokeDown(final Invoke x, final long pageId, final long backId, final long fwdId, final int lvl)
+        throws IgniteCheckedException {
+        assert lvl >= 0 : lvl;
+
+        if (x.isTail(pageId, lvl))
+            return FOUND; // We've already locked this page, so return that we are ok.
+
+        final Page page = page(pageId);
+
+        try {
+            for (;;) {
+                // Init args.
+                x.pageId = pageId;
+                x.fwdId = fwdId;
+                x.backId = backId;
+
+                Result res = readPage(page, this, search, x, lvl, RETRY);
+
+                switch (res) {
+                    case GO_DOWN_X:
+                        assert backId != 0;
+                        assert x.backId == 0; // We did not setup it yet.
+
+                        x.backId = pageId; // Dirty hack to setup a check inside of askNeighbor.
+
+                        // We need to get backId here for our child page, it must be the last child of our back.
+                        res = askNeighbor(backId, x, true);
+
+                        if (res != FOUND)
+                            return res; // Retry.
+
+                        assert x.backId != pageId; // It must be updated in askNeighbor.
+
+                        // Intentional fallthrough.
+                    case GO_DOWN:
+                        res = invokeDown(x, x.pageId, x.backId, x.fwdId, lvl - 1);
+
+                        if (res == RETRY) {
+                            checkInterrupted();
+
+                            continue;
+                        }
+
+                        if (res != RETRY_ROOT) {
+                            switch (x.finishIfNeeded()) {
+                                case DONE:
+                                    return res;
+
+                                case
+
+
+                            }
+                        }
+
+                        return res;
+
+                    case NOT_FOUND:
+                        // We are at the bottom.
+                        assert lvl == 0 : lvl;
+
+                        r.finish();
+
+                        return res;
+
+                    case FOUND:
+                        // We must be at the bottom here, just need to remove row from the current page.
+                        assert lvl == 0 : lvl;
+
+                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
+
+                        if (res == FOUND && r.tail == null) {
+                            // Finish if we don't need to do any merges.
+                            r.finish();
+                        }
+
+                        return res;
+
+                    default:
+                        return res;
+                }
+            }
+        }
+        finally {
+            x.levelExit();
+
+            if (x.canRelease(page, lvl))
+                page.close();
+        }
+    }
+
+
+    /**
      * @param row Lookup row.
      * @param needOld {@code True} if need return removed row.
      * @return Removed row.
@@ -1460,9 +1590,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             throw new AssertionError("Assertion error on search row: " + row, e);
         }
         finally {
-            r.releaseTail();
-
-            r.reuseFreePages();
+            r.releaseAll();
         }
     }
 
@@ -2059,6 +2187,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
+         * @param g Other operation to copy from.
+         */
+        final void copyFrom(Get g) {
+            rmvId = g.rmvId;
+            rootLvl = g.rootLvl;
+            row = g.row;
+            pageId = g.pageId;
+            fwdId = g.fwdId;
+            backId = g.backId;
+            shift = g.shift;
+        }
+
+        /**
          * Initialize operation.
          *
          * @throws IgniteCheckedException If failed.
@@ -2434,6 +2575,134 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
+     * Invoke operation.
+     */
+    private final class Invoke extends Get {
+        /** */
+        private final InvokeClosure<T> clo;
+
+        /** */
+        private boolean closureInvoked;
+
+        /** */
+        private Get op;
+
+        /**
+         * @param row Row.
+         * @param clo Closure.
+         */
+        private Invoke(L row, final InvokeClosure<T> clo) {
+            super(row);
+
+            this.clo = clo;
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
+            if (lvl == 0) {
+                invokeClosure(io, pageAddr, idx);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
+            if (lvl == 0) {
+                invokeClosure(null, 0L, 0);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /**
+         * @param io IO.
+         * @param pageAddr Page address.
+         * @param idx Index of found entry.
+         * @throws IgniteCheckedException If failed.
+         */
+        private void invokeClosure(BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException {
+            if (closureInvoked)
+                return;
+
+            boolean rowFound = io != null;
+
+            clo.call(rowFound ? getRow(io, pageAddr, idx) : null);
+
+            closureInvoked = true;
+
+            switch (clo.operationType()) {
+                case PUT:
+                    T newRow = clo.newRow();
+
+                    assert newRow != null;
+
+                    op = new Put(newRow, false);
+
+                    break;
+
+                case REMOVE:
+                    op = new Remove(row, false);
+
+                    break;
+
+                case NOOP:
+                    break;
+
+                default:
+                    throw new IllegalStateException();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean canRelease(Page page, int lvl) {
+            if (page == null)
+                return false;
+
+            if (op == null)
+                return true;
+
+            return op.canRelease(page, lvl);
+        }
+
+        /**
+         * @return {@code true} If the operation was finished successfully before this call or right now.
+         */
+        private Bool finishIfNeeded() {
+            // TODO
+            return DONE;
+        }
+
+        /**
+         * @param pageId Page ID.
+         * @param lvl Level.
+         * @return {@code true} If it is a {@link Remove} and the page is in tail.
+         */
+        private boolean isTail(long pageId, int lvl) {
+            return op != null && op.getClass() == Remove.class && ((Remove)op).isTail(pageId, lvl);
+        }
+
+        /**
+         */
+        private void levelExit() {
+            if (op != null && op.getClass() == Remove.class)
+                ((Remove)op).page = null;
+        }
+
+        /**
+         * Release all the resources by the end of operation.
+         */
+        private void releaseAll() throws IgniteCheckedException {
+            if (op != null && op.getClass() == Remove.class)
+                ((Remove)op).releaseAll();
+        }
+    }
+
+    /**
      * Remove operation.
      */
     private final class Remove extends Get implements ReuseBag {
@@ -3363,6 +3632,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         public boolean checkTailLevel(int rootLvl) {
             return tail == null || tail.lvl < rootLvl;
         }
+
+        /**
+         * @throws IgniteCheckedException If failed.
+         */
+        public void releaseAll() throws IgniteCheckedException {
+            releaseTail();
+            reuseFreePages();
+        }
     }
 
     /**
@@ -3946,21 +4223,4 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         DONE
     }
-
-    /**
-     * Type of invoke.
-     */
-    enum InvokeType {
-        /** */
-        INSERT,
-
-        /** */
-        REPLACE,
-
-        /** */
-        REMOVE,
-
-        /** */
-        NOOP
-    }
 }


[10/17] ignite git commit: ignite-db-x - invoke

Posted by se...@apache.org.
ignite-db-x - invoke


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/c16109c6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c16109c6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c16109c6

Branch: refs/heads/ignite-4652
Commit: c16109c62caaa3db9bb58b9562c96a07a406d076
Parents: 9417412
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed Feb 15 16:45:31 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Feb 15 16:45:31 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 154 ++++++++++++-------
 1 file changed, 102 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c16109c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index c21032f..347ee85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -991,7 +991,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             case RETRY:
                                 checkInterrupted();
 
-                                continue; // The child page got splitted, need to reread our page.
+                                continue; // The child page got split, need to reread our page.
 
                             default:
                                 return res;
@@ -1409,8 +1409,6 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** {@inheritDoc} */
     @Override public void invoke(L row, InvokeClosure<T> c) throws IgniteCheckedException {
-        assert c != null;
-
         checkDestroyed();
 
         Invoke x = new Invoke(row, c);
@@ -1419,7 +1417,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             for (;;) {
                 x.init();
 
-                switch (invokeDown(x, x.rootId, 0L, 0L, x.rootLvl)) {
+                Result res = invokeDown(x, x.rootId, 0L, 0L, x.rootLvl);
+
+                switch (res) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1427,8 +1427,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         continue;
 
                     default:
-                        if (x.finishIfNeeded() != FALSE)
-                            return;
+                        if (!x.isFinished()) {
+                            res = x.tryFinish();
+
+                            if (res == RETRY || res == RETRY_ROOT) {
+                                checkInterrupted();
+
+                                continue;
+                            }
+
+                            assert x.isFinished(): res;
+                        }
+
+                        return;
                 }
             }
         }
@@ -1490,7 +1501,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         // Intentional fallthrough.
                     case GO_DOWN:
-                        res = invokeDown(x, x.pageId, x.backId, x.fwdId, lvl - 1);
+                        res = x.tryReplaceInner(page, pageId, fwdId, lvl);
+
+                        if (res != RETRY)
+                            res = invokeDown(x, x.pageId, x.backId, x.fwdId, lvl - 1);
 
                         if (res == RETRY) {
                             checkInterrupted();
@@ -1498,37 +1512,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             continue;
                         }
 
-                        if (res != RETRY_ROOT) {
-                            switch (x.finishIfNeeded()) {
-                                case DONE:
-                                    return res;
-
-                                    // TODO
-                            }
-                        }
+                        if (res != RETRY_ROOT)
+                            res = x.onUp();
 
                         return res;
 
                     case NOT_FOUND:
-                        // We are at the bottom.
-//                        assert lvl == 0 : lvl;
-//
-//                        r.finish();
-
-                        return res;
+                        return x.onNotFound();
 
                     case FOUND:
-                        // We must be at the bottom here, just need to remove row from the current page.
-//                        assert lvl == 0 : lvl;
-//
-//                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
-//
-//                        if (res == FOUND && r.tail == null) {
-//                            // Finish if we don't need to do any merges.
-//                            r.finish();
-//                        }
-
-                        return res;
+                        return x.onFound();
 
                     default:
                         return res;
@@ -1782,7 +1775,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * {@inheritDoc}
      */
     @Override public final T put(T row) throws IgniteCheckedException {
-        return put(row, true);
+        return doPut(row, true);
     }
 
     /**
@@ -1791,7 +1784,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return {@code True} if replaced existing row.
      */
     public boolean putx(T row) throws IgniteCheckedException {
-        Boolean res = (Boolean)put(row, false);
+        Boolean res = (Boolean)doPut(row, false);
 
         return res != null ? res : false;
     }
@@ -1802,7 +1795,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return Old row.
      * @throws IgniteCheckedException If failed.
      */
-    private T put(T row, boolean needOld) throws IgniteCheckedException {
+    private T doPut(T row, boolean needOld) throws IgniteCheckedException {
         checkDestroyed();
 
         Put p = new Put(row, needOld);
@@ -2264,6 +2257,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         void fwdId(long fwdId) {
             this.fwdId = fwdId;
         }
+
+        /**
+         * @return {@code true} If the operation is finished.
+         */
+        boolean isFinished() {
+            throw new IllegalStateException();
+        }
     }
 
     /**
@@ -2425,10 +2425,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             tail(null, 0L);
         }
 
-        /**
-         * @return {@code true} If finished.
-         */
-        private boolean isFinished() {
+        /** {@inheritDoc} */
+        @Override boolean isFinished() {
             return row == null;
         }
 
@@ -2669,6 +2667,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Invoke(L row, final InvokeClosure<T> clo) {
             super(row);
 
+            assert clo != null;
+
             this.clo = clo;
         }
 
@@ -2790,12 +2790,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return op.canRelease(page, lvl);
         }
 
-        /**
-         * @return {@code true} If the operation was finished successfully before this call or right now.
-         */
-        private Bool finishIfNeeded() {
-            // TODO
-            return DONE;
+        private boolean is(Class<?> cls) {
+            return op != null && op.getClass() == cls;
         }
 
         /**
@@ -2804,13 +2800,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return {@code true} If it is a {@link Remove} and the page is in tail.
          */
         private boolean isTail(long pageId, int lvl) {
-            return op != null && op.getClass() == Remove.class && ((Remove)op).isTail(pageId, lvl);
+            return is(Remove.class) && ((Remove)op).isTail(pageId, lvl);
         }
 
         /**
          */
         private void levelExit() {
-            if (op != null && op.getClass() == Remove.class)
+            if (is(Remove.class))
                 ((Remove)op).page = null;
         }
 
@@ -2818,9 +2814,65 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * Release all the resources by the end of operation.
          */
         private void releaseAll() throws IgniteCheckedException {
-            if (op != null && op.getClass() == Remove.class)
+            if (is(Remove.class))
                 ((Remove)op).releaseAll();
         }
+
+        private Result onUp() {
+            return null; // TODO
+        }
+
+        private Result onNotFound() {
+            return null; // TODO
+        }
+
+        private Result onFound() {
+            return null; // TODO
+        }
+
+        /**
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Result tryFinish() throws IgniteCheckedException {
+            if (is(Put.class))
+                return RETRY;
+
+            Result res = ((Remove)op).finishTail();
+
+            if (res == NOT_FOUND)
+                res = RETRY;
+
+            assert res == FOUND || res == RETRY: res;
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean isFinished() {
+            if (!closureInvoked)
+                return false;
+
+            if (op == null)
+                return true;
+
+            return op.isFinished();
+        }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+            if (op == null || op.getClass() != Put.class)
+                return NOT_FOUND;
+
+            return ((Put)op).tryReplaceInner(page, pageId, fwdId, lvl);
+        }
     }
 
     /**
@@ -3592,10 +3644,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return true;
         }
 
-        /**
-         * @return {@code true} If finished.
-         */
-        private boolean isFinished() {
+        /** {@inheritDoc} */
+        @Override boolean isFinished() {
             return row == null;
         }
 


[12/17] ignite git commit: ignite-4652 - onFound/onNotFound

Posted by se...@apache.org.
ignite-4652 - onFound/onNotFound


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b4283a33
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b4283a33
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b4283a33

Branch: refs/heads/ignite-4652
Commit: b4283a330ecf02f11c39cf1f2a239333240e1d3f
Parents: e52a08e
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed Feb 15 19:13:23 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Feb 15 19:13:23 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 96 +++++++++++++++-----
 1 file changed, 74 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b4283a33/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 93097ba..570569f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -1509,23 +1509,27 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         if (res == RETRY_ROOT || x.isFinished())
                             return res;
 
-                        if (res == RETRY || x.isPut()) {
+                        if (res == RETRY) {
                             checkInterrupted();
 
                             continue;
                         }
 
-                        assert x.isRemove();
+                        // Put does insert on the same level.
+                        if (x.isPut())
+                            continue;
+
+                        assert x.isRemove(); // Guarded by isFinished.
 
                         res = x.finishOrLockTail(page, pageId, backId, fwdId, lvl);
 
                         return res;
 
                     case NOT_FOUND:
-                        return x.onNotFound();
+                        return x.onNotFound(page, pageId, fwdId, lvl);
 
                     case FOUND:
-                        return x.onFound();
+                        return x.onFound(page, pageId, backId, fwdId, lvl);
 
                     default:
                         return res;
@@ -1667,17 +1671,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         return res;
 
                     case FOUND:
-                        // We must be at the bottom here, just need to remove row from the current page.
-                        assert lvl == 0 : lvl;
-
-                        res = r.removeFromLeaf(pageId, page, backId, fwdId);
-
-                        if (res == FOUND && r.tail == null) {
-                            // Finish if we don't need to do any merges.
-                            r.finish();
-                        }
-
-                        return res;
+                        return r.tryRemoveFromLeaf(page, pageId, backId, fwdId, lvl);
 
                     default:
                         return res;
@@ -2830,12 +2824,48 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 ((Remove)op).releaseAll();
         }
 
-        private Result onNotFound() {
-            return null; // TODO
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Result onNotFound(Page page, long pageId, long fwdId, int lvl)
+            throws IgniteCheckedException {
+            if (op == null)
+                return NOT_FOUND;
+
+            if (isRemove()) {
+                assert lvl == 0;
+
+                ((Remove)op).finish();
+
+                return NOT_FOUND;
+            }
+
+            return ((Put)op).tryInsert(page, pageId, fwdId, lvl);
         }
 
-        private Result onFound() {
-            return null; // TODO
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param backId Back page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Result onFound(Page page, long pageId, long backId, long fwdId, int lvl)
+            throws IgniteCheckedException {
+            if (op == null)
+                return FOUND;
+
+            if (isRemove())
+                return ((Remove)op).tryRemoveFromLeaf(page, pageId, backId, fwdId, lvl);
+
+            return  ((Put)op).tryReplace(page, pageId, fwdId, lvl);
         }
 
         /**
@@ -3824,14 +3854,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @param rootLvl Actual root level.
          * @return {@code true} If tail level is correct.
          */
-        public boolean checkTailLevel(int rootLvl) {
+        private boolean checkTailLevel(int rootLvl) {
             return tail == null || tail.lvl < rootLvl;
         }
 
         /**
          * @throws IgniteCheckedException If failed.
          */
-        public void releaseAll() throws IgniteCheckedException {
+        private void releaseAll() throws IgniteCheckedException {
             releaseTail();
             reuseFreePages();
         }
@@ -3845,7 +3875,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        public Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             Result res = finishTail();
 
@@ -3854,6 +3884,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             return res;
         }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param backId Back page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Result tryRemoveFromLeaf(Page page, long pageId, long backId, long fwdId, int lvl)
+            throws IgniteCheckedException {
+            // We must be at the bottom here, just need to remove row from the current page.
+            assert lvl == 0 : lvl;
+
+            Result res = removeFromLeaf(pageId, page, backId, fwdId);
+
+            if (res == FOUND && tail == null) // Finish if we don't need to do any merges.
+                finish();
+
+            return res;
+        }
     }
 
     /**


[03/17] ignite git commit: Merge branches 'ignite-3477' and 'ignite-4652' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-4652

Posted by se...@apache.org.
Merge branches 'ignite-3477' and 'ignite-4652' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-4652


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d58d0f10
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d58d0f10
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d58d0f10

Branch: refs/heads/ignite-4652
Commit: d58d0f10c537d0c00131cdaa1acfee9a10e30294
Parents: 164995f 5bde9b1 5f83c94
Author: Sergi Vladykin <se...@gmail.com>
Authored: Thu Feb 9 16:19:58 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Thu Feb 9 16:19:58 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/PageMemory.java     |    4 -
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |   22 +-
 .../internal/pagemem/wal/record/DataRecord.java |   10 +-
 .../processors/cache/GridCacheMapEntry.java     | 1647 +++++++++++-------
 .../cache/GridCacheUpdateAtomicResult.java      |   96 +-
 .../cache/IgniteCacheOffheapManager.java        |   42 +
 .../cache/IgniteCacheOffheapManagerImpl.java    |  266 ++-
 .../cache/database/CacheDataRowAdapter.java     |   13 +
 .../cache/database/MetadataStorage.java         |   42 -
 .../cache/database/freelist/FreeListImpl.java   |    5 +-
 .../cache/database/tree/io/BPlusIO.java         |   12 -
 .../distributed/dht/GridDhtCacheEntry.java      |    5 +
 .../apache/ignite/internal/util/GridUnsafe.java |   45 +-
 .../GridCacheRebalancingSyncSelfTest.java       |   12 +-
 .../processors/database/BPlusTreeSelfTest.java  |   11 -
 .../unsafe/GridUnsafeMemorySelfTest.java        |   35 +
 .../junits/common/GridCommonAbstractTest.java   |    7 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   10 -
 .../testsuites/IgniteCacheTestSuite5.java       |   19 +-
 .../query/h2/database/io/H2InnerIO.java         |   10 -
 .../query/h2/database/io/H2LeafIO.java          |   10 -
 21 files changed, 1438 insertions(+), 885 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d58d0f10/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index e1ea6d1,531665c,eed9f09..77923c6
--- 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
@@@@ -954,40 -953,40 -1042,65 +1041,65 @@@@ public class IgniteCacheOffheapManagerI
                       }
                       else
                           old = dataTree.put(dataRow);
-- 
--                     if (old == null)
--                         storageSize.incrementAndGet();
                   }
   
--                 if (indexingEnabled) {
--                     GridCacheQueryManager qryMgr = cctx.queries();
++                 finishUpdate(dataRow, old);
++             }
++             finally {
++                 busyLock.leaveBusy();
++             }
++         }
 + 
-                      assert qryMgr.enabled();
++         /**
++          * @param newRow New row.
++          * @param oldRow Old row if available.
++          * @throws IgniteCheckedException If failed.
++          */
++         private void finishUpdate(CacheDataRow newRow, @Nullable CacheDataRow oldRow) throws IgniteCheckedException {
++             if (oldRow == null)
++                 storageSize.incrementAndGet();
   
-                      if (old != null)
-                          qryMgr.store(key, p, old.value(), old.version(), val, ver, expireTime, dataRow.link());
-                      else
-                          qryMgr.store(key, p, null, null, val, ver, expireTime, dataRow.link());
-                  }
 -                     assert qryMgr.enabled();
++             KeyCacheObject key = newRow.key();
   
-                  if (old != null) {
-                      assert old.link() != 0 : old;
 -                     if (old != null)
 -                         qryMgr.store(key, p, old.value(), old.version(), val, ver, expireTime, dataRow.link());
 -                     else
 -                         qryMgr.store(key, p, null, null, val, ver, expireTime, dataRow.link());
 -                 }
++             long expireTime = newRow.expireTime();
   
-                      if (pendingEntries != null && old.expireTime() != 0)
-                          pendingEntries.removex(new PendingRow(old.expireTime(), old.link()));
 -                 if (old != null) {
 -                     assert old.link() != 0 : old;
++             if (indexingEnabled) {
++                 GridCacheQueryManager qryMgr = cctx.queries();
   
-                      if (rmvOld)
-                          rowStore.removeRow(old.link());
 -                     if (pendingEntries != null && old.expireTime() != 0)
 -                         pendingEntries.removex(new PendingRow(old.expireTime(), old.link()));
++                 assert qryMgr.enabled();
+  
 -                     if (rmvOld)
 -                         rowStore.removeRow(old.link());
++                 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 (pendingEntries != null && expireTime != 0)
--                     pendingEntries.putx(new PendingRow(expireTime, dataRow.link()));
++             if (oldRow != null) {
++                 assert oldRow.link() != 0 : oldRow;
   
--                 updateIgfsMetrics(key, (old != null ? old.value() : null), val);
--             }
--             finally {
--                 busyLock.leaveBusy();
++                 if (pendingEntries != null && oldRow.expireTime() != 0)
++                     pendingEntries.removex(new PendingRow(oldRow.expireTime(), oldRow.link()));
++ 
++                 if (newRow.link() != oldRow.link())
++                     rowStore.removeRow(oldRow.link());
               }
++ 
++             if (pendingEntries != null && expireTime != 0)
++                 pendingEntries.putx(new PendingRow(expireTime, newRow.link()));
++ 
++             updateIgfsMetrics(key, (oldRow != null ? oldRow.value() : null), newRow.value());
           }
   
           /** {@inheritDoc} */
@@@@ -996,40 -995,40 -1109,49 +1108,49 @@@@
                   throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
   
               try {
--                 CacheDataRow dataRow = dataTree.remove(new SearchRow(key));
 - 
 -                 CacheObject val = null;
 -                 GridCacheVersion ver = null;
++                 CacheDataRow oldRow = dataTree.remove(new SearchRow(key));
   
-                  CacheObject val = null;
-                  GridCacheVersion ver = null;
-  
--                 if (dataRow != null) {
--                     assert dataRow.link() != 0 : dataRow;
++                 finishRemove(key, oldRow);
++             }
++             finally {
++                 busyLock.leaveBusy();
++             }
++         }
   
--                     if (pendingEntries != null && dataRow.expireTime() != 0)
--                         pendingEntries.removex(new PendingRow(dataRow.expireTime(), dataRow.link()));
++         /**
++          * @param key Key.
++          * @param oldRow Removed row.
++          * @throws IgniteCheckedException If failed.
++          */
++         private void finishRemove(KeyCacheObject key, @Nullable CacheDataRow oldRow) throws IgniteCheckedException {
++             CacheObject val = null;
++             GridCacheVersion ver = null;
   
--                     storageSize.decrementAndGet();
++             if (oldRow != null) {
++                 assert oldRow.link() != 0 : oldRow;
   
--                     val = dataRow.value();
++                 if (pendingEntries != null && oldRow.expireTime() != 0)
++                     pendingEntries.removex(new PendingRow(oldRow.expireTime(), oldRow.link()));
   
--                     ver = dataRow.version();
--                 }
++                 storageSize.decrementAndGet();
   
--                 if (indexingEnabled) {
--                     GridCacheQueryManager qryMgr = cctx.queries();
++                 val = oldRow.value();
   
--                     assert qryMgr.enabled();
++                 ver = oldRow.version();
++             }
   
--                     qryMgr.remove(key, partId, val, ver);
--                 }
++             if (indexingEnabled) {
++                 GridCacheQueryManager qryMgr = cctx.queries();
   
--                 if (dataRow != null)
--                     rowStore.removeRow(dataRow.link());
++                 assert qryMgr.enabled();
   
--                 updateIgfsMetrics(key, (dataRow != null ? dataRow.value() : null), null);
--             }
--             finally {
--                 busyLock.leaveBusy();
++                 qryMgr.remove(key, partId, val, ver);
               }
++ 
++             if (oldRow != null)
++                 rowStore.removeRow(oldRow.link());
++ 
++             updateIgfsMetrics(key, (oldRow != null ? oldRow.value() : null), null);
           }
   
           /** {@inheritDoc} */


[06/17] ignite git commit: ignite-db-x - tryReplaceInner

Posted by se...@apache.org.
ignite-db-x - tryReplaceInner


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/97e9bd96
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/97e9bd96
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/97e9bd96

Branch: refs/heads/ignite-4652
Commit: 97e9bd96028651e8deb4873c71a7a91cd40b91c4
Parents: 7689b36
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Feb 13 17:57:13 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Mon Feb 13 17:57:13 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 63 ++++++++++++--------
 1 file changed, 39 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/97e9bd96/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index e126923..f87ba2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -2055,31 +2055,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         assert p.pageId != pageId;
                         assert p.fwdId != fwdId || fwdId == 0;
 
-                        // Need to replace key in inner page. There is no race because we keep tail lock after split.
-                        if (p.needReplaceInner == TRUE) {
-                            p.needReplaceInner = FALSE; // Protect from retries.
+                        res = p.tryReplaceInner(page, pageId, fwdId, lvl);
 
-                            long oldFwdId = p.fwdId;
-                            long oldPageId = p.pageId;
-
-                            // Set old args.
-                            p.fwdId = fwdId;
-                            p.pageId = pageId;
-
-                            res = writePage(pageMem, page, this, replace, p, lvl, RETRY);
-
-                            // Restore args.
-                            p.pageId = oldPageId;
-                            p.fwdId = oldFwdId;
-
-                            if (res != FOUND)
-                                return res; // Need to retry.
-
-                            p.needReplaceInner = DONE; // We can have only single matching inner key.
-                        }
-
-                        // Go down recursively.
-                        res = putDown(p, p.pageId, p.fwdId, lvl - 1);
+                        if (res != RETRY) // Go down recursively.
+                            res = putDown(p, p.pageId, p.fwdId, lvl - 1);
 
                         if (res == RETRY_ROOT || p.isFinished())
                             return res;
@@ -2595,6 +2574,42 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 }
             }
         }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl)
+            throws IgniteCheckedException {
+            // Need to replace key in inner page. There is no race because we keep tail lock after split.
+            if (needReplaceInner == TRUE) {
+                needReplaceInner = FALSE; // Protect from retries.
+
+                long oldFwdId = this.fwdId;
+                long oldPageId = this.pageId;
+
+                // Set old args.
+                this.fwdId = fwdId;
+                this.pageId = pageId;
+
+                Result res = writePage(pageMem, page, BPlusTree.this, replace, this, lvl, RETRY);
+
+                // Restore args.
+                this.pageId = oldPageId;
+                this.fwdId = oldFwdId;
+
+                if (res == RETRY)
+                    return res;
+
+                needReplaceInner = DONE; // We can have only a single matching inner key.
+            }
+
+            return FOUND;
+        }
     }
 
     /**


[15/17] ignite git commit: ignite-4652 - invoke tests

Posted by se...@apache.org.
ignite-4652 - invoke tests


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/00820a94
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/00820a94
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/00820a94

Branch: refs/heads/ignite-4652
Commit: 00820a94e02ab8991a4eb6f3f4719be7eed692e5
Parents: a6e2569
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed Feb 15 20:34:14 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Feb 15 20:34:14 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          |   3 +
 .../processors/database/BPlusTreeSelfTest.java  | 153 +++++++++++++++++--
 2 files changed, 144 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/00820a94/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 5971844..20860f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -2757,6 +2757,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                     assert newRow != null;
 
+                    // Row key must be equal to the old one.
+                    assert !rowFound || compare(io, pageAddr, idx, newRow) == 0;
+
                     op = new Put(newRow, false);
 
                     break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/00820a94/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 2d9c693..e0a521a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -50,17 +50,22 @@ import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.GridStripedLock;
+import org.apache.ignite.internal.util.IgniteTree;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.ConcurrentLinkedHashMap;
 
 import static org.apache.ignite.internal.pagemem.PageIdUtils.effectivePageId;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.rnd;
+import static org.apache.ignite.internal.util.IgniteTree.OperationType.NOOP;
+import static org.apache.ignite.internal.util.IgniteTree.OperationType.PUT;
+import static org.apache.ignite.internal.util.IgniteTree.OperationType.REMOVE;
 
 /**
  */
@@ -561,6 +566,122 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     /**
      * @throws IgniteCheckedException If failed.
      */
+    public void testRandomInvoke_1_30_1() throws IgniteCheckedException {
+        MAX_PER_PAGE = 1;
+        CNT = 30;
+
+        doTestRandomInvoke(true);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testRandomInvoke_1_30_0() throws IgniteCheckedException {
+        MAX_PER_PAGE = 1;
+        CNT = 30;
+
+        doTestRandomInvoke(false);
+    }
+
+    /**
+     * @param canGetRow Can get row from inner page.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void doTestRandomInvoke(boolean canGetRow) throws IgniteCheckedException {
+        TestTree tree = createTestTree(canGetRow);
+
+        Map<Long,Long> map = new HashMap<>();
+
+        int loops = reuseList == null ? 100_000 : 300_000;
+
+        for (int i = 0 ; i < loops; i++) {
+            final Long x = (long)BPlusTree.randomInt(CNT);
+            final int rnd = BPlusTree.randomInt(11);
+
+            // Update map.
+            if (!map.containsKey(x)) {
+                if (rnd % 2 == 0) {
+                    map.put(x, x);
+
+                    X.println("put0: " + x);
+                }
+                else {
+                    X.println("noop0: " + x);
+                }
+            }
+            else {
+                if (rnd % 2 == 0) {
+                    X.println("put1: " + x);
+                }
+                else if (rnd % 3 == 0) {
+                    map.remove(x);
+
+                    X.println("rmv1: " + x);
+                }
+                else {
+                    X.println("noop1: " + x);
+                }
+            }
+
+            // Consistently update tree.
+            tree.invoke(x, new IgniteTree.InvokeClosure<Long>() {
+
+                IgniteTree.OperationType op;
+
+                Long newRow;
+
+                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                    if (row == null) {
+                        if (rnd % 2 == 0) {
+                            op = PUT;
+                            newRow = x;
+                        }
+                        else {
+                            op = NOOP;
+                            newRow = null;
+                        }
+                    }
+                    else {
+                        assertEquals(x, row);
+
+                        if (rnd % 2 == 0) {
+                            op = PUT;
+                            newRow = x; // We can not replace x with y here, because keys must be equal.
+                        }
+                        else if (rnd % 3 == 0) {
+                            op = REMOVE;
+                            newRow = null;
+                        }
+                        else {
+                            op = NOOP;
+                            newRow = null;
+                        }
+                    }
+                }
+
+                @Override public Long newRow() {
+                    return newRow;
+                }
+
+                @Override public IgniteTree.OperationType operationType() {
+                    return op;
+                }
+            });
+
+            assertNoLocks();
+
+            X.println(tree.printTree());
+
+            tree.validateTree();
+
+//            if (i % 100 == 0)
+                assertEqualContents(tree, map);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
     public void testRandomPutRemove_1_30_0() throws IgniteCheckedException {
         MAX_PER_PAGE = 1;
         CNT = 30;
@@ -840,24 +961,32 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 //            X.println(tree.printTree());
             tree.validateTree();
 
-            if (i % 100 == 0) {
-                GridCursor<Long> cursor = tree.find(null, null);
-
-                while (cursor.next()) {
-                    x = cursor.get();
+            if (i % 100 == 0)
+                assertEqualContents(tree, map);
+        }
+    }
 
-                    assert x != null;
+    /**
+     * @param tree Tree.
+     * @param map Map.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void assertEqualContents(IgniteTree<Long, Long> tree, Map<Long,Long> map) throws IgniteCheckedException {
+        GridCursor<Long> cursor = tree.find(null, null);
 
-                    assertEquals(map.get(x), x);
+        while (cursor.next()) {
+            Long x = cursor.get();
 
-                    assertNoLocks();
-                }
+            assert x != null;
 
-                assertEquals(map.size(), tree.size());
+            assertEquals(map.get(x), x);
 
-                assertNoLocks();
-            }
+            assertNoLocks();
         }
+
+        assertEquals(map.size(), tree.size());
+
+        assertNoLocks();
     }
 
     /**


[16/17] ignite git commit: Merge branch 'ignite-4652' of https://github.com/svladykin/ignite into ignite-4652

Posted by se...@apache.org.
Merge branch 'ignite-4652' of https://github.com/svladykin/ignite into ignite-4652


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e03af959
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e03af959
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e03af959

Branch: refs/heads/ignite-4652
Commit: e03af959c53416ed830f1c1cab3618cf20a6e654
Parents: 8b68231 00820a9
Author: Sergi Vladykin <se...@gmail.com>
Authored: Thu Feb 16 15:03:03 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Thu Feb 16 15:03:03 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 903 ++++++++++++++-----
 .../processors/database/BPlusTreeSelfTest.java  | 153 +++-
 2 files changed, 807 insertions(+), 249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e03af959/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index cdceffb,20860f5..c1589a6
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@@ -362,18 -355,27 +355,29 @@@ public abstract class BPlusTree<L, T ex
                  // Get old row in leaf page to reduce contention at upper level.
                  p.oldRow = p.needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
  
-                 p.finish();
- 
                  // Inner replace state must be consistent by the end of the operation.
                  assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
+ 
+                 // Need to replace inner key if now we are replacing the rightmost row and have a forward page.
+                 if (canGetRowFromInner && idx + 1 == cnt && p.fwdId != 0L && p.needReplaceInner == FALSE) {
+                     // Can happen only for invoke, otherwise inner key must be replaced on the way down.
+                     assert p.invoke;
+ 
+                     // We need to restart the operation from root to perform inner replace.
+                     // On the second pass we will not get here (will avoid infinite loop) because
+                     // needReplaceInner will be DONE or our key will not be the rightmost anymore.
+                     return RETRY_ROOT;
+                 }
+                 else
+                     p.finish();
              }
  
 -            io.store(pageAddr, idx, newRow, null);
 +            boolean needWal = needWalDeltaRecord(page);
  
 -            if (needWalDeltaRecord(page))
 -                wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRow, null, idx));
 +            byte[] newRowBytes = io.store(pageAddr, idx, newRow, null, needWal);
 +
 +            if (needWal)
 +                wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRowBytes, idx));
  
              return FOUND;
          }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e03af959/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------


[08/17] ignite git commit: ignite-db-x - wip

Posted by se...@apache.org.
ignite-db-x - wip


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4f1088fd
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4f1088fd
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4f1088fd

Branch: refs/heads/ignite-4652
Commit: 4f1088fd7ae130b17ab7f88f6411db7692adaf93
Parents: 9fe01a4
Author: Sergi Vladykin <se...@gmail.com>
Authored: Tue Feb 14 15:22:37 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Tue Feb 14 15:22:37 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 119 +++++++++++++------
 1 file changed, 81 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4f1088fd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 7770c27..51d2c83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -355,10 +355,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Get old row in leaf page to reduce contention at upper level.
                 p.oldRow = p.needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
 
-                p.finish();
-
                 // Inner replace state must be consistent by the end of the operation.
                 assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
+
+                p.finish();
             }
 
             io.store(pageAddr, idx, newRow, null);
@@ -2071,21 +2071,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     case FOUND: // Do replace.
                         assert lvl == 0 : "This replace can happen only at the bottom level.";
 
-                        // Init args.
-                        p.pageId = pageId;
-                        p.fwdId = fwdId;
-
-                        return writePage(pageMem, page, this, replace, p, lvl, RETRY);
+                        return p.tryReplace(page, pageId, fwdId, lvl);
 
                     case NOT_FOUND: // Do insert.
                         assert lvl == p.btmLvl : "must insert at the bottom level";
                         assert p.needReplaceInner == FALSE : p.needReplaceInner + " " + lvl;
 
-                        // Init args.
-                        p.pageId = pageId;
-                        p.fwdId = fwdId;
-
-                        return writePage(pageMem, page, this, insert, p, lvl, RETRY);
+                        return p.tryInsert(page, pageId, fwdId, lvl);
 
                     default:
                         return res;
@@ -2133,29 +2125,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private abstract class Get {
         /** */
-        protected long rmvId;
+        long rmvId;
 
         /** Starting point root level. May be outdated. Must be modified only in {@link Get#init()}. */
-        protected int rootLvl;
+        int rootLvl;
 
         /** Starting point root ID. May be outdated. Must be modified only in {@link Get#init()}. */
-        protected long rootId;
+        long rootId;
 
         /** */
-        protected L row;
+        L row;
 
         /** In/Out parameter: Page ID. */
-        protected long pageId;
+        long pageId;
 
         /** In/Out parameter: expected forward page ID. */
-        protected long fwdId;
+        long fwdId;
 
         /** In/Out parameter: in case of right turn this field will contain backward page ID for the child. */
-        protected long backId;
+        long backId;
 
         /** */
         int shift;
 
+        /** If {@code true}, then this operation is a part of invoke. */
+        boolean invoke;
+
         /**
          * @param row Row.
          */
@@ -2268,7 +2263,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private final class GetOne extends Get {
         /** */
-        private final RowClosure<L, ?> c;
+        final RowClosure<L, ?> c;
 
         /**
          * @param row Row.
@@ -2298,7 +2293,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private final class GetCursor extends Get {
         /** */
-        private ForwardCursor cursor;
+        ForwardCursor cursor;
 
         /**
          * @param lower Lower bound.
@@ -2335,31 +2330,31 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private final class Put extends Get {
         /** Right child page ID for split row. */
-        private long rightId;
+        long rightId;
 
         /** Replaced row if any. */
-        private T oldRow;
+        T oldRow;
 
         /**
          * This page is kept locked after split until insert to the upper level will not be finished.
          * It is needed because split row will be "in flight" and if we'll release tail, remove on
          * split row may fail.
          */
-        private Page tail;
+        Page tail;
 
         /** */
-        private long tailPageAddr;
+        long tailPageAddr;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
          */
-        private short btmLvl;
+        short btmLvl;
 
         /** */
         Bool needReplaceInner = FALSE;
 
         /** */
-        private final boolean needOld;
+        final boolean needOld;
 
         /**
          * @param row Row.
@@ -2606,9 +2601,43 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res;
 
                 needReplaceInner = DONE; // We can have only a single matching inner key.
+
+                return FOUND;
             }
 
-            return FOUND;
+            return NOT_FOUND;
+        }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Result tryInsert(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+            // Init args.
+            this.pageId = pageId;
+            this.fwdId = fwdId;
+
+            return writePage(pageMem, page, BPlusTree.this, insert, this, lvl, RETRY);
+        }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        public Result tryReplace(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+            // Init args.
+            this.pageId = pageId;
+            this.fwdId = fwdId;
+
+            return writePage(pageMem, page, BPlusTree.this, replace, this, lvl, RETRY);
         }
     }
 
@@ -2617,13 +2646,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private final class Invoke extends Get {
         /** */
-        private final InvokeClosure<T> clo;
+        final InvokeClosure<T> clo;
 
         /** */
-        private boolean closureInvoked;
+        boolean closureInvoked;
 
         /** */
-        private Get op;
+        Get op;
 
         /**
          * @param row Row.
@@ -2669,6 +2698,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /** {@inheritDoc} */
         @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
+            // If the operation is initialized, then the closure has been called already.
+            if (op != null)
+                return op.found(io, pageAddr, idx, lvl);
+
             if (lvl == 0) {
                 invokeClosure(io, pageAddr, idx);
 
@@ -2680,6 +2713,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /** {@inheritDoc} */
         @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
+            // If the operation is initialized, then the closure has been called already.
+            if (op != null)
+                return op.notFound(io, pageAddr, idx, lvl);
+
             if (lvl == 0) {
                 invokeClosure(null, 0L, 0);
 
@@ -2711,12 +2748,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                     assert newRow != null;
 
-                    op = new Put(newRow, false).copyFrom(this);
+                    op = new Put(newRow, false);
 
                     break;
 
                 case REMOVE:
-                    op = new Remove(row, false).copyFrom(this);
+                    op = new Remove(row, false);
 
                     break;
 
@@ -2726,6 +2763,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 default:
                     throw new IllegalStateException();
             }
+
+            if (op != null) {
+                op.copyFrom(this);
+
+                op.invoke = true;
+            }
         }
 
         /** {@inheritDoc} */
@@ -2777,7 +2820,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private final class Remove extends Get implements ReuseBag {
         /** We may need to lock part of the tree branch from the bottom to up for multiple levels. */
-        private Tail<L> tail;
+        Tail<L> tail;
 
         /** */
         Bool needReplaceInner = FALSE;
@@ -2786,16 +2829,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         Bool needMergeEmptyBranch = FALSE;
 
         /** Removed row. */
-        private T rmvd;
+        T rmvd;
 
         /** Current page. */
-        private Page page;
+        Page page;
 
         /** */
-        private Object freePages;
+        Object freePages;
 
         /** */
-        private final boolean needOld;
+        final boolean needOld;
 
         /**
          * @param row Row.


[14/17] ignite git commit: ignite-4652 - b+tree finished

Posted by se...@apache.org.
ignite-4652 - b+tree finished


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a6e2569e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a6e2569e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a6e2569e

Branch: refs/heads/ignite-4652
Commit: a6e2569e0834593ff79e8522a203e9a97d0bcf96
Parents: 607e418
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed Feb 15 19:36:03 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Feb 15 19:36:03 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/database/tree/BPlusTree.java    | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a6e2569e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 904d91e..5971844 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -363,7 +363,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     // Can happen only for invoke, otherwise inner key must be replaced on the way down.
                     assert p.invoke;
 
-                    p.needReplaceInner = TRUE;
+                    // We need to restart the operation from root to perform inner replace.
+                    // On the second pass we will not get here (will avoid infinite loop) because
+                    // needReplaceInner will be DONE or our key will not be the rightmost anymore.
+                    return RETRY_ROOT;
                 }
                 else
                     p.finish();
@@ -1515,7 +1518,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             continue;
                         }
 
-                        // Put does insert on the same level.
+                        // Unfinished Put does insertion on the same level.
                         if (x.isPut())
                             continue;
 


[11/17] ignite git commit: ignite-db-x - invoke2

Posted by se...@apache.org.
ignite-db-x - invoke2


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/e52a08ea
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/e52a08ea
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/e52a08ea

Branch: refs/heads/ignite-4652
Commit: e52a08ea293b19b6421df5ca24ce21c6cf0010ce
Parents: c16109c
Author: Sergi Vladykin <se...@gmail.com>
Authored: Wed Feb 15 17:54:12 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Feb 15 17:54:12 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 83 +++++++++++++++-----
 1 file changed, 63 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e52a08ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 347ee85..93097ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -1506,14 +1506,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         if (res != RETRY)
                             res = invokeDown(x, x.pageId, x.backId, x.fwdId, lvl - 1);
 
-                        if (res == RETRY) {
+                        if (res == RETRY_ROOT || x.isFinished())
+                            return res;
+
+                        if (res == RETRY || x.isPut()) {
                             checkInterrupted();
 
                             continue;
                         }
 
-                        if (res != RETRY_ROOT)
-                            res = x.onUp();
+                        assert x.isRemove();
+
+                        res = x.finishOrLockTail(page, pageId, backId, fwdId, lvl);
 
                         return res;
 
@@ -1647,12 +1651,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             continue;
                         }
 
-                        if (res != RETRY_ROOT && !r.isFinished()) {
-                            res = r.finishTail();
+                        if (res == RETRY_ROOT || r.isFinished())
+                            return res;
 
-                            if (res == NOT_FOUND)
-                                res = r.lockTail(pageId, page, backId, fwdId, lvl);
-                        }
+                        res = r.finishOrLockTail(page, pageId, backId, fwdId, lvl);
 
                         return res;
 
@@ -2604,7 +2606,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 this.fwdId = oldFwdId;
 
                 if (res == RETRY)
-                    return res;
+                    return RETRY;
 
                 needReplaceInner = DONE; // We can have only a single matching inner key.
 
@@ -2790,8 +2792,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return op.canRelease(page, lvl);
         }
 
-        private boolean is(Class<?> cls) {
-            return op != null && op.getClass() == cls;
+        /**
+         * @return {@code true} If it is a {@link Put} operation internally.
+         */
+        private boolean isPut() {
+            return op != null && op.getClass() == Put.class;
+        }
+
+        /**
+         * @return {@code true} If it is a {@link Remove} operation internally.
+         */
+        private boolean isRemove() {
+            return op != null && op.getClass() == Remove.class;
         }
 
         /**
@@ -2800,13 +2812,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return {@code true} If it is a {@link Remove} and the page is in tail.
          */
         private boolean isTail(long pageId, int lvl) {
-            return is(Remove.class) && ((Remove)op).isTail(pageId, lvl);
+            return isRemove() && ((Remove)op).isTail(pageId, lvl);
         }
 
         /**
          */
         private void levelExit() {
-            if (is(Remove.class))
+            if (isRemove())
                 ((Remove)op).page = null;
         }
 
@@ -2814,14 +2826,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * Release all the resources by the end of operation.
          */
         private void releaseAll() throws IgniteCheckedException {
-            if (is(Remove.class))
+            if (isRemove())
                 ((Remove)op).releaseAll();
         }
 
-        private Result onUp() {
-            return null; // TODO
-        }
-
         private Result onNotFound() {
             return null; // TODO
         }
@@ -2835,7 +2843,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private Result tryFinish() throws IgniteCheckedException {
-            if (is(Put.class))
+            assert op != null; // Must be guarded by isFinished.
+
+            if (isPut())
                 return RETRY;
 
             Result res = ((Remove)op).finishTail();
@@ -2868,11 +2878,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
-            if (op == null || op.getClass() != Put.class)
+            if (!isPut())
                 return NOT_FOUND;
 
             return ((Put)op).tryReplaceInner(page, pageId, fwdId, lvl);
         }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param backId Back page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        public Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+            throws IgniteCheckedException {
+            return ((Remove)op).finishOrLockTail(page, pageId, backId, fwdId, lvl);
+        }
     }
 
     /**
@@ -3811,6 +3835,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             releaseTail();
             reuseFreePages();
         }
+
+        /**
+         * @param page Page.
+         * @param pageId Page ID.
+         * @param backId Back page ID.
+         * @param fwdId Forward ID.
+         * @param lvl Level.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         */
+        public Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+            throws IgniteCheckedException {
+            Result res = finishTail();
+
+            if (res == NOT_FOUND)
+                res = lockTail(pageId, page, backId, fwdId, lvl);
+
+            return res;
+        }
     }
 
     /**


[17/17] ignite git commit: ignite-4652 - tests

Posted by se...@apache.org.
ignite-4652 - tests


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/affadf3b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/affadf3b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/affadf3b

Branch: refs/heads/ignite-4652
Commit: affadf3bb2a7278f46106c4e65c4b402e0b9aa80
Parents: e03af95
Author: Sergi Vladykin <se...@gmail.com>
Authored: Thu Feb 16 17:40:23 2017 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Thu Feb 16 17:40:23 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 27 ++++--
 .../processors/database/BPlusTreeSelfTest.java  | 96 +++++++++++++++++---
 2 files changed, 100 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/affadf3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index c1589a6..d35e7f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -350,10 +350,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Detach the old row if we are on a leaf page.
             if (lvl == 0) {
-                assert p.oldRow == null;
-
-                // Get old row in leaf page to reduce contention at upper level.
-                p.oldRow = p.needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
+                assert p.oldRow == null; // The old row must be set only once.
 
                 // Inner replace state must be consistent by the end of the operation.
                 assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
@@ -361,15 +358,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Need to replace inner key if now we are replacing the rightmost row and have a forward page.
                 if (canGetRowFromInner && idx + 1 == cnt && p.fwdId != 0L && p.needReplaceInner == FALSE) {
                     // Can happen only for invoke, otherwise inner key must be replaced on the way down.
-                    assert p.invoke;
+                    assert p.invoke != null;
 
                     // We need to restart the operation from root to perform inner replace.
                     // On the second pass we will not get here (will avoid infinite loop) because
                     // needReplaceInner will be DONE or our key will not be the rightmost anymore.
                     return RETRY_ROOT;
                 }
-                else
-                    p.finish();
+
+                // Get old row in leaf page to reduce contention at upper level.
+                p.oldRow = p.needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
+
+                p.finish();
             }
 
             boolean needWal = needWalDeltaRecord(page);
@@ -415,6 +415,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 p.btmLvl++; // Get high.
                 p.row = moveUpRow;
 
+                if (p.invoke != null)
+                    p.invoke.row = moveUpRow;
+
                 // Here forward page can't be concurrently removed because we keep write lock on tail which is the only
                 // page who knows about the forward page, because it was just produced by split.
                 p.rightId = io.getForward(pageAddr);
@@ -2196,8 +2199,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         int shift;
 
-        /** If {@code true}, then this operation is a part of invoke. */
-        boolean invoke;
+        /** If this operation is a part of invoke. */
+        Invoke invoke;
 
         /**
          * @param row Row.
@@ -2426,6 +2429,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (lvl == 0) // Leaf: need to stop.
                 return true;
 
+            assert btmLvl == 0; // It can not be insert.
+
             // If we can get full row from the inner page, we have to replace it with the new one. On the way down
             // we can not miss inner key even in presence of concurrent operations because of `triangle` invariant +
             // concurrent inner replace handling by retrying from root.
@@ -2816,6 +2821,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     break;
 
                 case REMOVE:
+                    assert rowFound;
+
                     op = new Remove(row, false);
 
                     break;
@@ -2830,7 +2837,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (op != null) {
                 op.copyFrom(this);
 
-                op.invoke = true;
+                op.invoke = this;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/affadf3b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index f450d7b..9db156d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -598,28 +598,33 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
             final Long x = (long)BPlusTree.randomInt(CNT);
             final int rnd = BPlusTree.randomInt(11);
 
+            if (i % 10_000 == 0) {
+//                X.println(tree.printTree());
+                X.println(" --> " + i + "  ++> " + x);
+            }
+
             // Update map.
             if (!map.containsKey(x)) {
                 if (rnd % 2 == 0) {
                     map.put(x, x);
 
-                    X.println("put0: " + x);
+//                    X.println("put0: " + x);
                 }
                 else {
-                    X.println("noop0: " + x);
+//                    X.println("noop0: " + x);
                 }
             }
             else {
                 if (rnd % 2 == 0) {
-                    X.println("put1: " + x);
+//                    X.println("put1: " + x);
                 }
                 else if (rnd % 3 == 0) {
                     map.remove(x);
 
-                    X.println("rmv1: " + x);
+//                    X.println("rmv1: " + x);
                 }
                 else {
-                    X.println("noop1: " + x);
+//                    X.println("noop1: " + x);
                 }
             }
 
@@ -670,11 +675,11 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
             assertNoLocks();
 
-            X.println(tree.printTree());
+//            X.println(tree.printTree());
 
             tree.validateTree();
 
-//            if (i % 100 == 0)
+            if (i % 100 == 0)
                 assertEqualContents(tree, map);
         }
     }
@@ -1168,27 +1173,28 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         final GridStripedLock lock = new GridStripedLock(256);
 
+        final String[] ops = {"put", "rmv", "inv_put", "inv_rmv"};
+
         IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 for (int i = 0; i < loops; i++) {
-                    Long x = (long)DataStructure.randomInt(CNT);
-
-                    boolean put = DataStructure.randomInt(2) == 0;
+                    final Long x = (long)DataStructure.randomInt(CNT);
+                    final int op = DataStructure.randomInt(4);
 
                     if (i % 10000 == 0)
-                        X.println(" --> " + (put ? "put " : "rmv ") + i + "  " + x);
+                        X.println(" --> " + ops[op] + "_" + i + "  " + x);
 
                     Lock l = lock.getLock(x.longValue());
 
                     l.lock();
 
                     try {
-                        if (put) {
+                        if (op == 0) { // Put.
                             assertEquals(map.put(x, x), tree.put(x));
 
                             assertNoLocks();
                         }
-                        else {
+                        else if (op == 1) { // Remove.
                             if (map.remove(x) != null) {
                                 assertEquals(x, tree.remove(x));
 
@@ -1199,6 +1205,54 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
                             assertNoLocks();
                         }
+                        else if (op == 2) {
+                            tree.invoke(x, new IgniteTree.InvokeClosure<Long>() {
+                                IgniteTree.OperationType opType;
+
+                                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                                    opType = PUT;
+
+                                    if (row != null)
+                                        assertEquals(x, row);
+                                }
+
+                                @Override public Long newRow() {
+                                    return x;
+                                }
+
+                                @Override public IgniteTree.OperationType operationType() {
+                                    return opType;
+                                }
+                            });
+
+                            map.put(x,x);
+                        }
+                        else if (op == 3) {
+                            tree.invoke(x, new IgniteTree.InvokeClosure<Long>() {
+                                IgniteTree.OperationType opType;
+
+                                @Override public void call(@Nullable Long row) throws IgniteCheckedException {
+                                    if (row != null) {
+                                        assertEquals(x, row);
+                                        opType = REMOVE;
+                                    }
+                                    else
+                                        opType = NOOP;
+                                }
+
+                                @Override public Long newRow() {
+                                    return null;
+                                }
+
+                                @Override public IgniteTree.OperationType operationType() {
+                                    return opType;
+                                }
+                            });
+
+                            map.remove(x);
+                        }
+                        else
+                            fail();
                     }
                     finally {
                         l.unlock();
@@ -1405,11 +1459,17 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onBeforeReadLock(Page page) {
+//            X.println("  onBeforeReadLock: " + U.hexLong(page.id()));
+//
+//            U.dumpStack();
+
             assertNull(beforeReadLock.put(threadId(), page.id()));
         }
 
         /** {@inheritDoc} */
         @Override public void onReadLock(Page page, long pageAddr) {
+//            X.println("  onReadLock: " + U.hexLong(page.id()));
+
             if (pageAddr != 0L) {
                 long pageId = PageIO.getPageId(pageAddr);
 
@@ -1423,6 +1483,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onReadUnlock(Page page, long pageAddr) {
+//            X.println("  onReadUnlock: " + U.hexLong(page.id()));
+
             checkPageId(page, pageAddr);
 
             long pageId = PageIO.getPageId(pageAddr);
@@ -1432,11 +1494,17 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onBeforeWriteLock(Page page) {
+//            X.println("  onBeforeWriteLock: " + U.hexLong(page.id()));
+
             assertNull(beforeWriteLock.put(threadId(), page.id()));
         }
 
         /** {@inheritDoc} */
         @Override public void onWriteLock(Page page, long pageAddr) {
+//            X.println("  onWriteLock: " + U.hexLong(page.id()));
+//
+//            U.dumpStack();
+
             if (pageAddr != 0L) {
                 checkPageId(page, pageAddr);
 
@@ -1453,6 +1521,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public void onWriteUnlock(Page page, long pageAddr) {
+//            X.println("  onWriteUnlock: " + U.hexLong(page.id()));
+
             assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
 
             assertEquals(Long.valueOf(page.id()), locks(false).remove(page.id()));