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/05 19:45:10 UTC

[8/9] ignite git commit: ignite-4652 - refactor b+tree for invoke

ignite-4652 - refactor b+tree for invoke


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

Branch: refs/heads/ignite-4652
Commit: 057617c23e5f235830dc0c14f9609082c54b85b1
Parents: 49787d7
Author: Sergi <se...@gmail.com>
Authored: Sat Feb 4 14:27:06 2017 +0300
Committer: Sergi <se...@gmail.com>
Committed: Sat Feb 4 14:27:06 2017 +0300

----------------------------------------------------------------------
 .../cache/database/tree/BPlusTree.java          | 127 ++++++++++++-------
 .../apache/ignite/internal/util/IgniteTree.java |   2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |   2 +-
 3 files changed, 84 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/057617c2/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 423acf2..a1860f0 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
@@ -78,6 +78,8 @@ 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;
+import static org.apache.ignite.internal.util.IgniteTree.OperationType.UPDATE;
 
 /**
  * Abstract B+Tree.
@@ -224,9 +226,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             long res = doAskNeighbor(io, pageAddr, back);
 
             if (back) {
-                assert g.getClass() == Remove.class;
+                assert g.getClass() == Invoke.class;
 
-                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
+                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in invokeDown for this check.
                     return RETRY;
 
                 g.backId = res;
@@ -418,14 +420,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> rmvFromLeaf = new RemoveFromLeaf();
+    private final GetPageHandler<Invoke> rmvFromLeaf = new RemoveFromLeaf();
 
     /**
      *
      */
-    private class RemoveFromLeaf extends GetPageHandler<Remove> {
+    private class RemoveFromLeaf extends GetPageHandler<Invoke> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
@@ -496,14 +498,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> lockBackAndRmvFromLeaf = new LockBackAndRmvFromLeaf();
+    private final GetPageHandler<Invoke> lockBackAndRmvFromLeaf = new LockBackAndRmvFromLeaf();
 
     /**
      *
      */
-    private class LockBackAndRmvFromLeaf extends GetPageHandler<Remove> {
+    private class LockBackAndRmvFromLeaf extends GetPageHandler<Invoke> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(pageAddr) != r.pageId)
@@ -521,14 +523,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> lockBackAndTail = new LockBackAndTail();
+    private final GetPageHandler<Invoke> lockBackAndTail = new LockBackAndTail();
 
     /**
      *
      */
-    private class LockBackAndTail extends GetPageHandler<Remove> {
+    private class LockBackAndTail extends GetPageHandler<Invoke> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(pageAddr) != r.pageId)
@@ -545,14 +547,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> lockTailForward = new LockTailForward();
+    private final GetPageHandler<Invoke> lockTailForward = new LockTailForward();
 
     /**
      *
      */
-    private class LockTailForward extends GetPageHandler<Remove> {
+    private class LockTailForward extends GetPageHandler<Invoke> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
             throws IgniteCheckedException {
             r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
 
@@ -561,14 +563,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> lockTail = new LockTail();
+    private final GetPageHandler<Invoke> lockTail = new LockTail();
 
     /**
      *
      */
-    private class LockTail extends GetPageHandler<Remove> {
+    private class LockTail extends GetPageHandler<Invoke> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Invoke r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
@@ -1424,15 +1426,36 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @throws IgniteCheckedException If failed.
      */
     private T doRemove(L row, boolean ceil, boolean needOld) throws IgniteCheckedException {
+        return doInvoke(row, ceil, needOld, null);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override public void invoke(L key, InvokeClosure<T> c) throws IgniteCheckedException {
+        assert c != null;
+
+        doInvoke(key, false, false, c);
+    }
+
+    /**
+     * @param row Lookup row.
+     * @param ceil If we can remove ceil row when we can not find exact.
+     * @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 ceil, boolean needOld, InvokeClosure<T> c) throws IgniteCheckedException {
         checkDestroyed();
 
-        Remove r = new Remove(row, ceil, needOld);
+        Invoke r = new Invoke(row, ceil, needOld, c);
 
         try {
             for (;;) {
                 r.init();
 
-                switch (removeDown(r, r.rootId, 0L, 0L, r.rootLvl)) {
+                switch (invokeDown(r, r.rootId, 0L, 0L, r.rootLvl)) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1445,11 +1468,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                             // If not found, then the tree grew beyond our call stack -> retry from the actual root.
                             if (res == RETRY || res == NOT_FOUND) {
-                                int root = getRootLevel();
-
-                                boolean checkRes = r.checkTailLevel(root);
-
-                                assert checkRes : "tail=" + r.tail + ", root=" + root + ", res=" + res;
+                                assert r.checkTailLevel(getRootLevel()) : "tail=" + r.tail + ", res=" + res;
 
                                 checkInterrupted();
 
@@ -1461,7 +1480,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         assert r.isFinished();
 
-                        return r.rmvd;
+                        return r.oldRow;
                 }
             }
         }
@@ -1490,7 +1509,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return Result code.
      * @throws IgniteCheckedException If failed.
      */
-    private Result removeDown(final Remove r, final long pageId, final long backId, final long fwdId, final int lvl)
+    private Result invokeDown(final Invoke r, final long pageId, final long backId, final long fwdId, final int lvl)
         throws IgniteCheckedException {
         assert lvl >= 0 : lvl;
 
@@ -1525,7 +1544,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         // Intentional fallthrough.
                     case GO_DOWN:
-                        res = removeDown(r, r.pageId, r.backId, r.fwdId, lvl - 1);
+                        res = invokeDown(r, r.pageId, r.backId, r.fwdId, lvl - 1);
 
                         if (res == RETRY) {
                             checkInterrupted();
@@ -1546,7 +1565,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         // We are at the bottom.
                         assert lvl == 0 : lvl;
 
-                        if (!r.ceil) {
+                        if (r.opType == UPDATE) {
+                            // TODO insert
+                        }
+                        else if (!r.ceil) {
                             r.finish();
 
                             return res;
@@ -1669,13 +1691,6 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * {@inheritDoc}
      */
-    @Override public void invoke(L key, InvokeClosure<L> c) throws IgniteCheckedException {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
     @Override public final T put(T row) throws IgniteCheckedException {
         return put(row, true);
     }
@@ -2466,7 +2481,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * Remove operation.
      */
-    private final class Remove extends Get implements ReuseBag {
+    private final class Invoke extends Get implements ReuseBag {
         /** */
         private boolean ceil;
 
@@ -2479,8 +2494,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         Bool needMergeEmptyBranch = FALSE;
 
-        /** Removed row. */
-        private T rmvd;
+        /** Updated or removed row. */
+        private T oldRow;
 
         /** Current page. */
         private Page page;
@@ -2491,28 +2506,39 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         private final boolean needOld;
 
+        /** */
+        private final InvokeClosure<T> c;
+
+        /** */
+        private OperationType opType;
+
         /**
          * @param row Row.
          * @param ceil If we can remove ceil row when we can not find exact.
          * @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 Remove(L row, boolean ceil, boolean needOld) {
+        private Invoke(L row, boolean ceil, boolean needOld, InvokeClosure<T> c) {
             super(row);
 
             this.ceil = ceil;
             this.needOld = needOld;
+            this.c = c;
+
+            if (c == null)
+                opType = REMOVE;
         }
 
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public long pollFreePage() {
             if (freePages == null)
-                return 0;
+                return 0L;
 
             if (freePages.getClass() == GridLongList.class) {
                 GridLongList list = ((GridLongList)freePages);
 
-                return list.isEmpty() ? 0 : list.remove();
+                return list.isEmpty() ? 0L : list.remove();
             }
 
             long res = (long)freePages;
@@ -2525,7 +2551,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public void addFreePage(long pageId) {
-            assert pageId != 0;
+            assert pageId != 0L;
 
             if (freePages == null)
                 freePages = pageId;
@@ -2546,16 +2572,27 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             if (lvl == 0) {
                 assert tail == null;
 
+                if (c != null)
+                    opType = c.call(null);
+
                 return true;
             }
 
             return false;
         }
 
+        /** {@inheritDoc} */
+        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
+            if (lvl == 0 && c != null)
+                opType = c.call(getRow(io, pageAddr, idx));
+
+            return super.found(io, pageAddr, idx, lvl);
+        }
+
         /**
          * Finish the operation.
          */
@@ -2644,7 +2681,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return {@code true} If already removed from leaf.
          */
         private boolean isRemoved() {
-            return rmvd != null;
+            return oldRow != null;
         }
 
         /**
@@ -2920,7 +2957,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert !isRemoved(): "already removed";
 
             // Detach the row.
-            rmvd = needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
+            oldRow = needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
 
             doRemove(page, io, pageAddr, cnt, idx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057617c2/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
index faf4034..7968361 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
@@ -39,7 +39,7 @@ public interface IgniteTree<L, T> {
      * @param c Closure.
      * @throws IgniteCheckedException If failed.
      */
-    public void invoke(L key, InvokeClosure<L> c) throws IgniteCheckedException;
+    public void invoke(L key, InvokeClosure<T> c) throws IgniteCheckedException;
 
     /**
      * Returns the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the

http://git-wip-us.apache.org/repos/asf/ignite/blob/057617c2/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 2b47733..68d3686 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -501,7 +501,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         /**
          * {@inheritDoc}
          */
-        @Override public void invoke(GridSearchRowPointer key, InvokeClosure<GridSearchRowPointer> c) {
+        @Override public void invoke(GridSearchRowPointer key, InvokeClosure<GridH2Row> c) {
             throw new UnsupportedOperationException();
         }