You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/01/13 08:12:40 UTC

ignite git commit: Do not read old value.

Repository: ignite
Updated Branches:
  refs/heads/ignite-gg-11810-2 [created] 922e4b4a3


Do not read old value.


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

Branch: refs/heads/ignite-gg-11810-2
Commit: 922e4b4a3d8205c7805dfb149c8f04bf25f14439
Parents: 81ce9d5
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jan 13 11:12:42 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jan 13 11:12:42 2017 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapManagerImpl.java    | 47 ++++++++------
 .../cache/database/tree/BPlusTree.java          | 65 ++++++++++++++++----
 2 files changed, 80 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/922e4b4a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 3a68b81..b1c31b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -899,7 +899,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                 assert dataRow.link() != 0 : dataRow;
 
-                CacheDataRow old = dataTree.put(dataRow);
+                Long old = dataTree.put(dataRow, CacheDataTree.linkC);
 
                 if (old == null)
                     storageSize.incrementAndGet();
@@ -909,23 +909,23 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                     assert qryMgr.enabled();
 
-                    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());
+//                    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());
                 }
 
                 if (old != null) {
-                    assert old.link() != 0 : old;
-
-                    if (pendingEntries != null && old.expireTime() != 0)
-                        pendingEntries.remove(new PendingRow(old.expireTime(), old.link()));
+//                    assert old.link() != 0 : old;
+//
+//                    if (pendingEntries != null && old.expireTime() != 0)
+//                        pendingEntries.remove(new PendingRow(old.expireTime(), old.link()));
 
-                    rowStore.removeRow(old.link());
+                    rowStore.removeRow(old);
                 }
 
                 if (pendingEntries != null && expireTime != 0)
-                    pendingEntries.put(new PendingRow(expireTime, dataRow.link()));
+                    pendingEntries.putx(new PendingRow(expireTime, dataRow.link()));
             }
             finally {
                 busyLock.leaveBusy();
@@ -1180,6 +1180,13 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** */
         private final GridCacheContext cctx;
 
+        /** */
+        private static final BPlusTree.RowClosure<CacheSearchRow, Long> linkC = new RowClosure<CacheSearchRow, Long>() {
+            @Override public Long oldRow(BPlusIO<CacheSearchRow> io, long pageAddr, int idx) throws IgniteCheckedException {
+                return ((RowLinkIO)io).getLink(pageAddr, idx);
+            }
+        };
+
         /**
          * @param name Tree name.
          * @param reuseList Reuse list.
@@ -1228,6 +1235,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             return compareKeys(row.key(), link);
         }
 
+        /** {@inheritDoc} */
+        @Override protected CacheDataRow getRow(BPlusIO<CacheSearchRow> io, long pageAddr, int idx)
+            throws IgniteCheckedException {
+            int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
+            long link = ((RowLinkIO)io).getLink(pageAddr, idx);
+
+            return rowStore.dataRow(hash, link);
+        }
+
         /**
          * @param key Key.
          * @param link Link.
@@ -1295,15 +1311,6 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
             return Integer.compare(bytes1.length, bytes2.length);
         }
-
-        /** {@inheritDoc} */
-        @Override protected CacheDataRow getRow(BPlusIO<CacheSearchRow> io, long pageAddr, int idx)
-            throws IgniteCheckedException {
-            int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
-            long link = ((RowLinkIO)io).getLink(pageAddr, idx);
-
-            return rowStore.dataRow(hash, link);
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/922e4b4a/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 e91dda9..2e4bead 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
@@ -116,6 +116,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     private final AtomicLong globalRmvId;
 
     /** */
+    private final RowClosure<L, T> oldRowC = new RowClosure<L, T>() {
+        @Override public T oldRow(BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException {
+            return getRow(io, pageAddr, idx);
+        }
+    };
+
+    public RowClosure<L, T> noOldValue() {
+        return (RowClosure)NO_OLD_VAL;
+    }
+
+    private static final BPlusTree.RowClosure<Object, Object> NO_OLD_VAL = new RowClosure<Object, Object>() {
+        @Override public Object oldRow(BPlusIO<Object> io, long pageAddr, int idx) throws IgniteCheckedException {
+            return null;
+        }
+    };
+
+    /** */
     private final GridTreePrinter<Long> treePrinter = new GridTreePrinter<Long>() {
         /** */
         private boolean keys = true;
@@ -317,7 +334,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 assert p.oldRow == null;
 
                 // Get old row in leaf page to reduce contention at upper level.
-                p.oldRow = getRow(io, pageAddr, idx);
+                p.oldRow = p.oldRowC.oldRow(io, pageAddr, idx);
 
                 p.finish();
 
@@ -1518,20 +1535,33 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * {@inheritDoc}
      */
     @Override public final T put(T row) throws IgniteCheckedException {
-        return put(row, null);
+        return put(row, (ReuseBag)null);
+    }
+
+    public void putx(T row) throws IgniteCheckedException {
+        put(row, noOldValue());
+    }
+
+    public final <R> R put(T row, RowClosure<L, R> c) throws IgniteCheckedException {
+        Put p = new Put(row, null, c);
+
+        return (R)put0(p);
+    }
+
+    public final T put(T row, ReuseBag bag) throws IgniteCheckedException {
+        Put p = new Put(row, bag, oldRowC);
+
+        return (T)put0(p);
     }
 
     /**
-     * @param row Row.
-     * @param bag Reuse bag.
+     * @param p Put operation.
      * @return Old row.
      * @throws IgniteCheckedException If failed.
      */
-    public final T put(T row, ReuseBag bag) throws IgniteCheckedException {
+    private Object put0(Put p) throws IgniteCheckedException {
         checkDestroyed();
 
-        Put p = new Put(row, bag);
-
         try {
             for (;;) { // Go down with retries.
                 p.init();
@@ -1566,13 +1596,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Runtime failure on row: " + row, e);
+            throw new IgniteCheckedException("Runtime failure on row: " + p.row, e);
         }
         catch (RuntimeException e) {
-            throw new IgniteException("Runtime failure on row: " + row, e);
+            throw new IgniteException("Runtime failure on row: " + p.row, e);
         }
         catch (AssertionError e) {
-            throw new AssertionError("Assertion error on row: " + row, e);
+            throw new AssertionError("Assertion error on row: " + p.row, e);
         }
         finally {
             p.releaseMeta();
@@ -2084,7 +2114,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private long rightId;
 
         /** Replaced row if any. */
-        private T oldRow;
+        private Object oldRow;
 
         /**
          * This page is kept locked after split until insert to the upper level will not be finished.
@@ -2107,14 +2137,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         private ReuseBag bag;
 
+        /** */
+        private final RowClosure<L, ?> oldRowC;
+
         /**
          * @param row Row.
          * @param bag Reuse bag.
          */
-        private Put(T row, ReuseBag bag) {
+        private Put(T row, ReuseBag bag, RowClosure<L, ?> oldRowC) {
             super(row);
 
             this.bag = bag;
+            this.oldRowC = oldRowC;
         }
 
         /** {@inheritDoc} */
@@ -3450,6 +3484,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     protected abstract T getRow(BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException;
 
     /**
+     *
+     */
+    public interface RowClosure<L, R> {
+        public R oldRow(BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException;
+    }
+
+    /**
      * Forward cursor.
      */
     @SuppressWarnings("unchecked")