You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dp...@apache.org on 2018/04/06 15:28:35 UTC

ignite git commit: IGNITE-8049 Limit the number of operation cycles in B+Tree - Fixes #3769.

Repository: ignite
Updated Branches:
  refs/heads/master 8053fc1c9 -> e491f109c


IGNITE-8049 Limit the number of operation cycles in B+Tree - Fixes #3769.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/master
Commit: e491f109c1a316f5b6dad1468f5b84054c278b3f
Parents: 8053fc1
Author: Алексей Стельмак <as...@MacBook-Pro-Aleksej.local>
Authored: Fri Apr 6 18:28:22 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Fri Apr 6 18:28:22 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  5 +++
 .../cache/persistence/tree/BPlusTree.java       | 41 +++++++++++++++++++-
 .../processors/database/BPlusTreeSelfTest.java  | 29 ++++++++++++++
 3 files changed, 74 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e491f109/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index f7128c0..152d845 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -845,6 +845,11 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_READ_LOAD_BALANCING = "IGNITE_READ_LOAD_BALANCING";
 
     /**
+     * Number of repetitions to capture a lock in the B+Tree.
+     */
+    public static final String IGNITE_BPLUS_TREE_LOCK_RETRIES = "IGNITE_BPLUS_TREE_LOCK_RETRIES";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e491f109/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index a520dce..4d05095 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -87,6 +88,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     private static volatile boolean interrupted;
 
     /** */
+    private static final int IGNITE_BPLUS_TREE_LOCK_RETRIES_DEFAULT = 1000;
+
+    /** */
+    private static final int LOCK_RETRIES = IgniteSystemProperties.getInteger(
+        IgniteSystemProperties.IGNITE_BPLUS_TREE_LOCK_RETRIES, IGNITE_BPLUS_TREE_LOCK_RETRIES_DEFAULT);
+
+    /** */
     private final AtomicBoolean destroyed = new AtomicBoolean(false);
 
     /** */
@@ -1123,6 +1131,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         try {
             for (;;) {
+                g.checkLockRetry();
+
                 // Init args.
                 g.pageId = pageId;
                 g.fwdId = fwdId;
@@ -1657,13 +1667,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long page = acquirePage(pageId);
 
         try {
+            Result res = RETRY;
+
             for (;;) {
+                if (res == RETRY)
+                    x.checkLockRetry();
+
                 // Init args.
                 x.pageId(pageId);
                 x.fwdId(fwdId);
                 x.backId(backId);
 
-                Result res = read(pageId, page, search, x, lvl, RETRY);
+                res = read(pageId, page, search, x, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN_X:
@@ -1813,6 +1828,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         try {
             for (;;) {
+                r.checkLockRetry();
+
                 // Init args.
                 r.pageId = pageId;
                 r.fwdId = fwdId;
@@ -2310,6 +2327,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         try {
             for (;;) {
+                p.checkLockRetry();
+
                 // Init args.
                 p.pageId = pageId;
                 p.fwdId = fwdId;
@@ -2422,6 +2441,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** Ignore row passed, find last row */
         boolean findLast;
 
+        /** Number of repetitions to capture a lock in the B+Tree (countdown). */
+        int lockRetriesCnt = getLockRetries();
+
         /**
          * @param row Row.
          * @param findLast find last row.
@@ -2534,6 +2556,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         boolean isFinished() {
             throw new IllegalStateException();
         }
+
+        /**
+         * @throws IgniteCheckedException If the operation can not be retried.
+         */
+        final void checkLockRetry() throws IgniteCheckedException {
+            if (lockRetriesCnt == 0)
+                throw new IgniteCheckedException("Maximum of retries " + getLockRetries() + " reached.");
+
+            lockRetriesCnt--;
+        }
     }
 
     /**
@@ -4911,4 +4943,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         public boolean apply(BPlusTree<L, T> tree, BPlusIO<L> io, long pageAddr, int idx)
             throws IgniteCheckedException;
     }
+
+    /**
+     * @return Return number of retries.
+     */
+    protected int getLockRetries() {
+        return LOCK_RETRIES;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e491f109/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 72d2e0f..83d0ddd 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
@@ -230,6 +230,26 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     /**
      * @throws IgniteCheckedException If failed.
      */
+    public void testRetries() throws IgniteCheckedException {
+        TestTree tree = createTestTree(true);
+
+        tree.numRetries = 1;
+
+        long size = CNT * CNT;
+
+        try {
+            for (long i = 1; i <= size; i++)
+                tree.put(i);
+
+            fail();
+        }
+        catch (IgniteCheckedException ignored) {
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
     public void _testBenchInvoke() throws IgniteCheckedException {
         MAX_PER_PAGE = 10;
 
@@ -2349,6 +2369,9 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         /** */
         private static ConcurrentMap<Object, Map<Long, Long>> writeLocks = new ConcurrentHashMap<>();
 
+        /** Number of retries. */
+        private int numRetries = super.getLockRetries();
+
         /**
          * @param reuseList Reuse list.
          * @param canGetRow Can get row from inner page.
@@ -2536,6 +2559,12 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
             return b.toString();
         }
+
+        /** {@inheritDoc} */
+        @Override
+        protected int getLockRetries() {
+            return numRetries;
+        }
     }
 
     /**