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 2015/11/26 09:40:31 UTC

ignite git commit: ignite-1537

Repository: ignite
Updated Branches:
  refs/heads/ignite-1537 f70e862ce -> cadbddd9e


ignite-1537


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

Branch: refs/heads/ignite-1537
Commit: cadbddd9e10ad6119b59d375afb915852fb4a5fa
Parents: f70e862
Author: sboikov <sb...@gridgain.com>
Authored: Thu Nov 26 11:39:51 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 26 11:39:51 2015 +0300

----------------------------------------------------------------------
 ...arOptimisticSerializableTxPrepareFuture.java | 94 ++++----------------
 .../near/GridNearOptimisticTxPrepareFuture.java | 51 ++++-------
 ...ridNearOptimisticTxPrepareFutureAdapter.java | 75 ++++++++++++++++
 .../transactions/IgniteTxLocalAdapter.java      |  2 +-
 4 files changed, 111 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cadbddd9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index 144070c..02225c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -39,12 +39,10 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTx
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -76,10 +74,6 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
 
     /** */
     @GridToStringExclude
-    private KeyLockFuture keyLockFut = new KeyLockFuture();
-
-    /** */
-    @GridToStringExclude
     private ClientRemapFuture remapFut;
 
     /**
@@ -134,7 +128,8 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
                     }
                 }
 
-                keyLockFut.onKeyLocked(entry.txKey());
+                if (keyLockFut != null)
+                    keyLockFut.onKeyLocked(entry.txKey());
 
                 return true;
             }
@@ -189,7 +184,8 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
 
         err.compareAndSet(null, e);
 
-        keyLockFut.onDone(e);
+        if (keyLockFut != null)
+            keyLockFut.onDone(e);
     }
 
     /** {@inheritDoc} */
@@ -210,7 +206,8 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
         if (err != null) {
             this.err.compareAndSet(null, err);
 
-            keyLockFut.onDone(err);
+            if (keyLockFut != null)
+                keyLockFut.onDone(err);
         }
 
         return onComplete();
@@ -335,10 +332,8 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
         for (IgniteTxEntry read : reads)
             map(read, topVer, mappings, remap, topLocked);
 
-        keyLockFut.onAllKeysAdded();
-
-        if (!remap)
-            add(keyLockFut);
+        if (keyLockFut != null)
+            keyLockFut.onAllKeysAdded();
 
         if (isDone()) {
             if (log.isDebugEnabled())
@@ -535,8 +530,15 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
             entry.cached(cacheCtx.local().entryEx(entry.key(), topVer));
 
         if (!remap && (cacheCtx.isNear() || cacheCtx.isLocal())) {
-            if (entry.explicitVersion() == null)
+            if (entry.explicitVersion() == null) {
+                if (keyLockFut == null) {
+                    keyLockFut = new KeyLockFuture();
+
+                    add(keyLockFut);
+                }
+
                 keyLockFut.addLockKey(entry.txKey());
+            }
         }
 
         IgniteBiTuple<ClusterNode, Boolean> key = F.t(primary, cacheCtx.isNear());
@@ -854,68 +856,4 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
             return S.toString(MiniFuture.class, this, "done", isDone(), "cancelled", isCancelled(), "err", error());
         }
     }
-
-    /**
-     * Keys lock future.
-     */
-    private class KeyLockFuture extends GridFutureAdapter<GridNearTxPrepareResponse> {
-        /** */
-        @GridToStringInclude
-        private Collection<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
-
-        /** */
-        private volatile boolean allKeysAdded;
-
-        /**
-         * @param key Key to track for locking.
-         */
-        private void addLockKey(IgniteTxKey key) {
-            assert !allKeysAdded;
-
-            lockKeys.add(key);
-        }
-
-        /**
-         * @param key Locked keys.
-         */
-        private void onKeyLocked(IgniteTxKey key) {
-            lockKeys.remove(key);
-
-            checkLocks();
-        }
-
-        /**
-         * Moves future to the ready state.
-         */
-        private void onAllKeysAdded() {
-            allKeysAdded = true;
-
-            checkLocks();
-        }
-
-        /**
-         * @return {@code True} if all locks are owned.
-         */
-        private boolean checkLocks() {
-            boolean locked = lockKeys.isEmpty();
-
-            if (locked && allKeysAdded) {
-                if (log.isDebugEnabled())
-                    log.debug("All locks are acquired for near prepare future: " + this);
-
-                onDone((GridNearTxPrepareResponse)null);
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Still waiting for locks [fut=" + this + ", keys=" + lockKeys + ']');
-            }
-
-            return locked;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(KeyLockFuture.class, this, super.toString());
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cadbddd9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index e70e574..61f297e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -40,10 +40,8 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTx
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
-import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.C1;
@@ -64,10 +62,6 @@ import static org.apache.ignite.transactions.TransactionState.PREPARING;
  *
  */
 public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepareFutureAdapter {
-    /** */
-    @GridToStringInclude
-    private Collection<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
-
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -84,10 +78,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
             log.debug("Transaction future received owner changed callback: " + entry);
 
         if ((entry.context().isNear() || entry.context().isLocal()) && owner != null && tx.hasWriteKey(entry.txKey())) {
-            lockKeys.remove(entry.txKey());
-
-            // This will check for locks.
-            onDone();
+            if (keyLockFut != null)
+                keyLockFut.onKeyLocked(entry.txKey());
 
             return true;
         }
@@ -151,24 +143,6 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         }
     }
 
-    /**
-     * @return {@code True} if all locks are owned.
-     */
-    private boolean checkLocks() {
-        boolean locked = lockKeys.isEmpty();
-
-        if (locked) {
-            if (log.isDebugEnabled())
-                log.debug("All locks are acquired for near prepare future: " + this);
-        }
-        else {
-            if (log.isDebugEnabled())
-                log.debug("Still waiting for locks [fut=" + this + ", keys=" + lockKeys + ']');
-        }
-
-        return locked;
-    }
-
     /** {@inheritDoc} */
     @Override public void onResult(UUID nodeId, GridNearTxPrepareResponse res) {
         if (!isDone()) {
@@ -215,8 +189,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx t, Throwable err) {
-        // If locks were not acquired yet, delay completion.
-        if (isDone() || (err == null && !checkLocks()))
+        if (isDone())
             return false;
 
         this.err.compareAndSet(null, err);
@@ -320,6 +293,9 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
             return;
         }
 
+        if (keyLockFut != null)
+            keyLockFut.onAllKeysAdded();
+
         tx.addSingleEntryMapping(mapping, write);
 
         cctx.mvcc().recheckPendingLocks();
@@ -385,6 +361,9 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
             return;
         }
 
+        if (keyLockFut != null)
+            keyLockFut.onAllKeysAdded();
+
         tx.addEntryMapping(mappings);
 
         cctx.mvcc().recheckPendingLocks();
@@ -543,8 +522,15 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
             entry.cached(cacheCtx.local().entryEx(entry.key(), topVer));
 
         if (cacheCtx.isNear() || cacheCtx.isLocal()) {
-            if (entry.explicitVersion() == null)
-                lockKeys.add(entry.txKey());
+            if (entry.explicitVersion() == null) {
+                if (keyLockFut == null) {
+                    keyLockFut = new KeyLockFuture();
+
+                    add(keyLockFut);
+                }
+
+                keyLockFut.addLockKey(entry.txKey());
+            }
         }
 
         if (cur == null || !cur.node().id().equals(primary.id()) || cur.near() != cacheCtx.isNear()) {
@@ -598,6 +584,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
         return S.toString(GridNearOptimisticTxPrepareFuture.class, this,
             "innerFuts", futs,
+            "keyLockFut", keyLockFut,
             "tx", tx,
             "super", super.toString());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cadbddd9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
index 6b7244a..fd13f75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
@@ -17,20 +17,31 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import java.util.Collection;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridPlainRunnable;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 
 /**
  *
  */
 public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearTxPrepareFutureAdapter {
+    /** */
+    @GridToStringExclude
+    protected KeyLockFuture keyLockFut;
+
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -157,4 +168,68 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
      * @param topLocked {@code True} if thread already acquired lock preventing topology change.
      */
     protected abstract void prepare0(boolean remap, boolean topLocked);
+
+    /**
+     * Keys lock future.
+     */
+    protected class KeyLockFuture extends GridFutureAdapter<GridNearTxPrepareResponse> {
+        /** */
+        @GridToStringInclude
+        private Collection<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
+
+        /** */
+        private volatile boolean allKeysAdded;
+
+        /**
+         * @param key Key to track for locking.
+         */
+        protected void addLockKey(IgniteTxKey key) {
+            assert !allKeysAdded;
+
+            lockKeys.add(key);
+        }
+
+        /**
+         * @param key Locked keys.
+         */
+        protected void onKeyLocked(IgniteTxKey key) {
+            lockKeys.remove(key);
+
+            checkLocks();
+        }
+
+        /**
+         * Moves future to the ready state.
+         */
+        protected void onAllKeysAdded() {
+            allKeysAdded = true;
+
+            checkLocks();
+        }
+
+        /**
+         * @return {@code True} if all locks are owned.
+         */
+        private boolean checkLocks() {
+            boolean locked = lockKeys.isEmpty();
+
+            if (locked && allKeysAdded) {
+                if (log.isDebugEnabled())
+                    log.debug("All locks are acquired for near prepare future: " + this);
+
+                onDone((GridNearTxPrepareResponse)null);
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Still waiting for locks [fut=" + this + ", keys=" + lockKeys + ']');
+            }
+
+            return locked;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(KeyLockFuture.class, this, super.toString());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cadbddd9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 7c6a1d4..d74e80b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -598,7 +598,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
      * @param entry Cache entry to check.
      */
     private void checkCommitLocks(GridCacheEntryEx entry) {
-        assert ownsLockUnsafe(entry) : "Lock is not owned for commit in PESSIMISTIC mode [entry=" + entry +
+        assert ownsLockUnsafe(entry) : "Lock is not owned for commit [entry=" + entry +
             ", tx=" + this + ']';
     }