You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2014/12/13 04:21:30 UTC

[1/3] incubator-ignite git commit: GG-9141 - Fixed group lock transactions.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1 d7309f496 -> ee2df1d8d


GG-9141 - Fixed group lock transactions.


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

Branch: refs/heads/ignite-1
Commit: f60670a9263d9e4005b971b0d278e07fab2433b3
Parents: d7309f4
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Dec 12 18:43:20 2014 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Dec 12 18:43:20 2014 -0800

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteTransactions.java   | 10 +-
 .../processors/cache/GridCacheAdapter.java      | 97 +-------------------
 .../cache/GridCacheSharedContext.java           |  1 -
 .../processors/cache/GridCacheTxHandler.java    |  6 +-
 .../cache/GridCacheTxLocalAdapter.java          |  2 +-
 .../distributed/dht/GridDhtCacheEntryImpl.java  |  2 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  1 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  | 10 +-
 .../transactions/IgniteTransactionsImpl.java    | 86 +++++++++++++++--
 .../ipc/shmem/GridIpcSharedMemorySpace.java     |  1 -
 .../GridCacheGroupLockAbstractSelfTest.java     | 49 +++-------
 ...ockPartitionedMultiNodeAbstractSelfTest.java |  1 -
 12 files changed, 110 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
index 061ec38..d28954c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
@@ -9,7 +9,7 @@
 
 package org.apache.ignite;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.configuration.*;
 import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
@@ -64,7 +64,7 @@ public interface IgniteTransactions {
      * transaction. All updates to the keys involved should always go through {@code affinity-group-locked}
      * transaction, otherwise cache may be left in inconsistent state.
      * <p>
-     * If cache sanity check is enabled ({@link org.apache.ignite.configuration.IgniteConfiguration#isCacheSanityCheckEnabled()}),
+     * If cache sanity check is enabled ({@link IgniteConfiguration#isCacheSanityCheckEnabled()}),
      * the following checks are performed:
      * <ul>
      *     <li>
@@ -86,7 +86,7 @@ public interface IgniteTransactions {
      * @throws IgniteCheckedException If local node is not primary for any of provided keys.
      * @throws UnsupportedOperationException If cache is {@link GridCacheAtomicityMode#ATOMIC}.
      */
-    public GridCacheTx txStartAffinity(Object affinityKey, GridCacheTxConcurrency concurrency,
+    public GridCacheTx txStartAffinity(String cacheName, Object affinityKey, GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException, IgniteCheckedException;
 
     /**
@@ -100,7 +100,7 @@ public interface IgniteTransactions {
      * transaction. All updates to the keys involved should always go through {@code partition-group-locked}
      * transaction, otherwise, cache may be left in inconsistent state.
      * <p>
-     * If cache sanity check is enabled ({@link org.apache.ignite.configuration.IgniteConfiguration#isCacheSanityCheckEnabled()}),
+     * If cache sanity check is enabled ({@link IgniteConfiguration#isCacheSanityCheckEnabled()}),
      * the following checks are performed:
      * <ul>
      *     <li>
@@ -122,7 +122,7 @@ public interface IgniteTransactions {
      * @throws IgniteCheckedException If local node is not primary for any of provided keys.
      * @throws UnsupportedOperationException If cache is {@link GridCacheAtomicityMode#ATOMIC}.
      */
-    public GridCacheTx txStartPartition(int partId, GridCacheTxConcurrency concurrency,
+    public GridCacheTx txStartPartition(String cacheName, int partId, GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException, IgniteCheckedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
index bf40a85..4d136a1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
@@ -52,8 +52,6 @@ import static org.gridgain.grid.cache.GridCacheFlag.*;
 import static org.gridgain.grid.cache.GridCachePeekMode.*;
 import static org.gridgain.grid.cache.GridCacheTxConcurrency.*;
 import static org.gridgain.grid.cache.GridCacheTxIsolation.*;
-import static org.gridgain.grid.cache.GridCacheTxState.*;
-import static org.apache.ignite.events.IgniteEventType.*;
 import static org.gridgain.grid.kernal.GridClosureCallMode.*;
 import static org.gridgain.grid.kernal.processors.dr.GridDrType.*;
 import static org.gridgain.grid.kernal.processors.task.GridTaskThreadContextKey.*;
@@ -3138,106 +3136,21 @@ public abstract class GridCacheAdapter<K, V> extends GridMetadataAwareAdapter im
     /** {@inheritDoc} */
     @Override public GridCacheTx txStart(GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException {
-        A.notNull(concurrency, "concurrency");
-        A.notNull(isolation, "isolation");
-        A.ensure(timeout >= 0, "timeout cannot be negative");
-        A.ensure(txSize >= 0, "transaction size cannot be negative");
-
-        GridTransactionsConfiguration cfg = ctx.gridConfig().getTransactionsConfiguration();
-
-        if (!cfg.isTxSerializableEnabled() && isolation == SERIALIZABLE)
-            throw new IllegalArgumentException("SERIALIZABLE isolation level is disabled (to enable change " +
-                "'txSerializableEnabled' configuration property)");
-
-        GridCacheTxEx<K, V> tx = (GridCacheTxEx<K, V>)ctx.tm().userTx();
-
-        if (tx != null)
-            throw new IllegalStateException("Failed to start new transaction " +
-                "(current thread already has a transaction): " + tx);
-
-        tx = ctx.tm().newTx(
-            false,
-            false,
-            concurrency,
-            isolation,
-            timeout,
-            false,
-            txSize,
-            /** group lock keys */null,
-            /** partition lock */false
-        );
-
-        assert tx != null;
-
-        // Wrap into proxy.
-        return new GridCacheTxProxyImpl<>(tx, ctx.shared());
+        return ctx.kernalContext().cache().transactions().txStart(concurrency, isolation, timeout, txSize);
     }
 
     /** {@inheritDoc} */
     @Override public GridCacheTx txStartAffinity(Object affinityKey, GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException, IgniteCheckedException {
-        return txStartGroupLock(ctx.txKey((K)affinityKey), concurrency, isolation, false, timeout, txSize);
+        return ctx.kernalContext().cache().transactions().txStartAffinity(name(), affinityKey, concurrency, isolation,
+            timeout, txSize);
     }
 
     /** {@inheritDoc} */
     @Override public GridCacheTx txStartPartition(int partId, GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException, IgniteCheckedException {
-        Object grpLockKey = ctx.affinity().partitionAffinityKey(partId);
-
-        return txStartGroupLock(ctx.txKey((K)grpLockKey), concurrency, isolation, true, timeout, txSize);
-    }
-
-    /**
-     * Internal method to start group-lock transaction.
-     *
-     * @param grpLockKey Group lock key.
-     * @param concurrency Transaction concurrency control.
-     * @param isolation Transaction isolation level.
-     * @param partLock {@code True} if this is a partition-lock transaction. In this case {@code grpLockKey}
-     *      should be a unique partition-specific key.
-     * @param timeout Tx timeout.
-     * @param txSize Expected transaction size.
-     * @return Started transaction.
-     * @throws IllegalStateException If other transaction was already started.
-     * @throws IgniteCheckedException In case of error.
-     */
-    @SuppressWarnings("unchecked")
-    private GridCacheTx txStartGroupLock(GridCacheTxKey grpLockKey, GridCacheTxConcurrency concurrency,
-        GridCacheTxIsolation isolation, boolean partLock, long timeout, int txSize)
-        throws IllegalStateException, IgniteCheckedException {
-        GridCacheTx tx = ctx.tm().userTx();
-
-        if (tx != null)
-            throw new IllegalStateException("Failed to start new transaction " +
-                "(current thread already has a transaction): " + tx);
-
-        GridCacheTxLocalEx<K, V> tx0 = ctx.tm().newTx(
-            false,
-            false,
-            concurrency,
-            isolation,
-            timeout,
-            ctx.hasFlag(INVALIDATE),
-            txSize,
-            grpLockKey,
-            partLock
-        );
-
-        assert tx0 != null;
-
-        IgniteFuture<?> lockFut = tx0.groupLockAsync(ctx, (Collection)F.asList(grpLockKey));
-
-        try {
-            lockFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            tx0.rollback();
-
-            throw e;
-        }
-
-        // Wrap into proxy.
-        return new GridCacheTxProxyImpl<>(tx0, ctx.shared());
+        return ctx.kernalContext().cache().transactions().txStartPartition(name(), partId, concurrency, isolation,
+            timeout, txSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSharedContext.java
index 33ffea5..e67519b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSharedContext.java
@@ -14,7 +14,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.communication.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxHandler.java
index 85a240c..e629d3f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxHandler.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
@@ -1229,13 +1228,14 @@ public class GridCacheTxHandler<K, V> {
 
                 while (true) {
                     try {
-                        entry = cacheCtx.dht().near().peekExx(txEntry.key());
+                        entry = cacheCtx.near().peekExx(txEntry.key());
 
                         if (entry != null) {
                             entry.keyBytes(txEntry.keyBytes());
 
                             // Handle implicit locks for pessimistic transactions.
-                            tx = ctx.tm().tx(req.version());
+                            if (tx == null)
+                                tx = ctx.tm().nearTx(req.version());
 
                             if (tx == null) {
                                 tx = new GridNearTxRemote<>(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
index 4ac7b4e..2c52e0a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
@@ -2541,7 +2541,7 @@ public abstract class GridCacheTxLocalAdapter<K, V> extends GridCacheTxAdapter<K
                             ", part=" + part + ", groupLockKey=" + grpLockKey + ']');
                 }
                 else {
-                    Object affinityKey = cacheCtx.config().getAffinityMapper().affinityKey(key);
+                    GridCacheTxKey affinityKey = cacheCtx.txKey((K)cacheCtx.config().getAffinityMapper().affinityKey(key));
 
                     if (!grpLockKey.equals(affinityKey))
                         throw new IgniteCheckedException("Failed to enlist key into group-lock transaction (affinity key was " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
index 094df1b..c37fb90 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
@@ -60,7 +60,7 @@ public class GridDhtCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V> {
 
     /** {@inheritDoc} */
     @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws IgniteCheckedException {
-        if (!ctx.isNear() && !ctx.isReplicated() && modes.contains(NEAR_ONLY))
+        if (!ctx.isNear() && modes.contains(NEAR_ONLY))
             return null;
 
         V val = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 2e9311d..a836ff8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index d594df3..95cd4ad 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -635,10 +635,10 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends GridCacheTxLocalAdapte
         for (GridCacheTxKey<K> key : keys) {
             GridCacheTxEntry<K, V> txEntry = entry(key);
 
-            if (!txEntry.groupLockEntry())
+            if (!txEntry.groupLockEntry() || txEntry.context().isNear())
                 continue;
 
-            assert txEntry.cached() instanceof GridDhtCacheEntry;
+            assert txEntry.cached() instanceof GridDhtCacheEntry : "Invalid entry type: " + txEntry.cached();
 
             while (true) {
                 try {
@@ -674,10 +674,10 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends GridCacheTxLocalAdapte
                     txEntry.cached(txEntry.context().dht().entryExx(key.key(), topologyVersion()), txEntry.keyBytes());
                 }
             }
-
-            if (locNearMap != null)
-                addNearMapping(locNearMap);
         }
+
+        if (locNearMap != null)
+            addNearMapping(locNearMap);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTransactionsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTransactionsImpl.java
index 89bbb74..a8e1344 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTransactionsImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/transactions/IgniteTransactionsImpl.java
@@ -10,12 +10,16 @@
 package org.gridgain.grid.kernal.processors.cache.transactions;
 
 import org.apache.ignite.*;
-import org.gridgain.grid.*;
+import org.apache.ignite.lang.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
+import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
+import java.util.*;
+
+import static org.gridgain.grid.cache.GridCacheFlag.*;
 import static org.gridgain.grid.cache.GridCacheTxIsolation.*;
 
 /**
@@ -115,22 +119,86 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactions {
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheTx txStartAffinity(Object affinityKey, GridCacheTxConcurrency concurrency,
+    @Override public GridCacheTx txStartAffinity(String cacheName, Object affinityKey, GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException, IgniteCheckedException {
-        // TODO: implement.
-        return null;
+        GridCacheAdapter<Object, Object> cache = cctx.kernalContext().cache().internalCache(cacheName);
+
+        if (cache == null)
+            throw new IllegalArgumentException("Failed to find cache with given name (cache is not configured): " +
+                cacheName);
+
+        return txStartGroupLock(cache.context(), affinityKey, concurrency, isolation, false, timeout, txSize);
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheTx txStartPartition(int partId, GridCacheTxConcurrency concurrency,
+    @Override public GridCacheTx txStartPartition(String cacheName, int partId, GridCacheTxConcurrency concurrency,
         GridCacheTxIsolation isolation, long timeout, int txSize) throws IllegalStateException, IgniteCheckedException {
-        // TODO: implement.
-        return null;
+        GridCacheAdapter<Object, Object> cache = cctx.kernalContext().cache().internalCache(cacheName);
+
+        if (cache == null)
+            throw new IllegalArgumentException("Failed to find cache with given name (cache is not configured): " +
+                cacheName);
+
+        Object grpLockKey = cache.context().affinity().partitionAffinityKey(partId);
+
+        return txStartGroupLock(cache.context(), grpLockKey, concurrency, isolation, true, timeout, txSize);
+    }
+
+    /**
+     * Internal method to start group-lock transaction.
+     *
+     * @param grpLockKey Group lock key.
+     * @param concurrency Transaction concurrency control.
+     * @param isolation Transaction isolation level.
+     * @param partLock {@code True} if this is a partition-lock transaction. In this case {@code grpLockKey}
+     *      should be a unique partition-specific key.
+     * @param timeout Tx timeout.
+     * @param txSize Expected transaction size.
+     * @return Started transaction.
+     * @throws IllegalStateException If other transaction was already started.
+     * @throws IgniteCheckedException In case of error.
+     */
+    @SuppressWarnings("unchecked")
+    private GridCacheTx txStartGroupLock(GridCacheContext ctx, Object grpLockKey, GridCacheTxConcurrency concurrency,
+        GridCacheTxIsolation isolation, boolean partLock, long timeout, int txSize)
+        throws IllegalStateException, IgniteCheckedException {
+        GridCacheTx tx = cctx.tm().userTx();
+
+        if (tx != null)
+            throw new IllegalStateException("Failed to start new transaction " +
+                "(current thread already has a transaction): " + tx);
+
+        GridCacheTxLocalEx<K, V> tx0 = cctx.tm().newTx(
+            false,
+            false,
+            concurrency,
+            isolation,
+            timeout,
+            ctx.hasFlag(INVALIDATE),
+            txSize,
+            ctx.txKey(grpLockKey),
+            partLock
+        );
+
+        assert tx0 != null;
+
+        IgniteFuture<?> lockFut = tx0.groupLockAsync(ctx, (Collection)F.asList(grpLockKey));
+
+        try {
+            lockFut.get();
+        }
+        catch (IgniteCheckedException e) {
+            tx0.rollback();
+
+            throw e;
+        }
+
+        // Wrap into proxy.
+        return new GridCacheTxProxyImpl<>(tx0, cctx);
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public GridCacheTx tx() {
-        // TODO: implement.
-        return null;
+        return cctx.tm().userTx();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/main/java/org/gridgain/grid/util/ipc/shmem/GridIpcSharedMemorySpace.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/ipc/shmem/GridIpcSharedMemorySpace.java b/modules/core/src/main/java/org/gridgain/grid/util/ipc/shmem/GridIpcSharedMemorySpace.java
index 6d81561..d25338b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/ipc/shmem/GridIpcSharedMemorySpace.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/ipc/shmem/GridIpcSharedMemorySpace.java
@@ -10,7 +10,6 @@
 package org.gridgain.grid.util.ipc.shmem;
 
 import org.apache.ignite.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.io.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
index f9fe5e9..34cd126 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
@@ -13,7 +13,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.cache.store.*;
@@ -73,8 +73,6 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
         cacheCfg.setWriteSynchronizationMode(GridCacheWriteSynchronizationMode.FULL_SYNC);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
 
-        boolean txBatchUpdate = batchUpdate();
-
         cacheCfg.setStore(store);
 
         cfg.setCacheConfiguration(cacheCfg);
@@ -86,6 +84,12 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
         cfg.setDiscoverySpi(discoSpi);
 
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
         return cfg;
     }
 
@@ -797,8 +801,8 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
         }
 
         for (int i = 0; i < gridCount(); i++) {
-            assertNull("For cache: " + i, cache(i).peek("val1"));
-            assertNull("For cache: " + i, cache(i).peek("val2"));
+            assertNull("For cache [i=" + i + ", val=" + cache(i).peek(key1) + ']', cache(i).peek(key1));
+            assertNull("For cache [i=" + i + ", val=" + cache(i).peek(key2) + ']', cache(i).peek(key2));
 
             assertTrue("For cache [idx=" + i + ", keySet=" + cache(i).keySet() + ']', cache(i).size() <= 1);
         }
@@ -911,7 +915,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
         cache(0).put(key, "val");
 
-        try (GridCacheTx tx = cache(0).txStartPartition(cache(0).affinity().partition(key), concurrency,
+        try (GridCacheTx ignored = cache(0).txStartPartition(cache(0).affinity().partition(key), concurrency,
             REPEATABLE_READ, 0, 1)) {
             assertEquals("val", cache(0).get(key));
         }
@@ -939,7 +943,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
         final GridCache<GridCacheAffinityKey<String>, String> cache = grid(0).cache(null);
 
-        try (GridCacheTx tx = cache.txStartAffinity(affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
+        try (GridCacheTx ignored = cache.txStartAffinity(affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
             // Key with affinity key different from enlisted on tx start should raise exception.
             cache.put(new GridCacheAffinityKey<>("key1", UUID.randomUUID()), "val1");
 
@@ -978,7 +982,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
         cache.put(key, "val");
 
-        try (GridCacheTx tx = cache.txStartAffinity(affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
+        try (GridCacheTx ignored = cache.txStartAffinity(affinityKey, concurrency, READ_COMMITTED, 0, 1)) {
             // Key with affinity key different from enlisted on tx start should raise exception.
             cache.remove(key);
 
@@ -1062,9 +1066,6 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
      * @throws Exception If failed.
      */
     public void testGroupLockWriteThroughBatchUpdateOptimistic() throws Exception {
-        // Configuration changed according to test name.
-        assert batchUpdate();
-
         checkGroupLockWriteThrough(OPTIMISTIC);
     }
 
@@ -1072,25 +1073,6 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
      * @throws Exception If failed.
      */
     public void testGroupLockWriteThroughBatchUpdatePessimistic() throws Exception {
-        // Configuration changed according to test name.
-        assert batchUpdate();
-
-        checkGroupLockWriteThrough(PESSIMISTIC);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGroupLockWriteThroughSingleUpdateOptimistic() throws Exception {
-        // Configuration changed according to test name.
-        checkGroupLockWriteThrough(OPTIMISTIC);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGroupLockWriteThroughSingleUpdatePessimistic() throws Exception {
-        // Configuration changed according to test name.
         checkGroupLockWriteThrough(PESSIMISTIC);
     }
 
@@ -1142,12 +1124,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
 
         // Check the store.
         assertTrue(store.storeMap().equals(putMap));
-        assertEquals(batchUpdate() ? 1 : 4, store.putCount());
-    }
-
-    /** @return {@code True} if batch update should be enabled. */
-    private boolean batchUpdate() {
-        return getName().contains("testGroupLockWriteThroughBatchUpdate");
+        assertEquals(1, store.putCount());
     }
 
     /** @return {@code True} if sanity check should be enabled. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f60670a9/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGroupLockPartitionedMultiNodeAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGroupLockPartitionedMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGroupLockPartitionedMultiNodeAbstractSelfTest.java
index 12a7cbd..aa1d4ad 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGroupLockPartitionedMultiNodeAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGroupLockPartitionedMultiNodeAbstractSelfTest.java
@@ -10,7 +10,6 @@
 package org.gridgain.grid.kernal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.util.typedef.*;


[3/3] incubator-ignite git commit: GG-9141 - Fixing queries and enabling passing tests.

Posted by ag...@apache.org.
GG-9141 - Fixing queries and enabling passing tests.


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

Branch: refs/heads/ignite-1
Commit: ee2df1d8d23256be0b911849878c652d0eda6076
Parents: 59d9cd6
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Dec 12 19:21:15 2014 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Dec 12 19:21:15 2014 -0800

----------------------------------------------------------------------
 .../grid/kernal/processors/cache/GridCacheMapEntry.java        | 4 ++--
 .../deployment/GridDeploymentMessageCountSelfTest.java         | 2 +-
 .../GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java   | 2 +-
 .../cache/eviction/GridCacheDistributedEvictionsSelfTest.java  | 1 -
 .../cache/eviction/GridCacheEvictionTouchSelfTest.java         | 4 ++--
 .../eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java   | 2 +-
 .../continuous/GridCacheContinuousQueryAbstractSelfTest.java   | 1 -
 .../gridgain/testsuites/GridCacheEvictionSelfTestSuite.java    | 2 +-
 .../org/gridgain/testsuites/GridCacheQuerySelfTestSuite.java   | 6 +++---
 9 files changed, 11 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
index 32c1485..eb131fd 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
@@ -1212,7 +1212,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
             GridCacheMode mode = cctx.config().getCacheMode();
 
             if (mode == GridCacheMode.LOCAL || mode == GridCacheMode.REPLICATED ||
-                (tx != null && (tx.dht() || tx.colocated()) && tx.local()))
+                (tx != null && tx.local() && !isNear()))
                 cctx.continuousQueries().onEntryUpdate(this, key, val, valueBytesUnlocked(), old, oldBytes);
 
             cctx.dataStructures().onEntryUpdated(key, false);
@@ -1366,7 +1366,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
                 GridCacheMode mode = cctx.config().getCacheMode();
 
                 if (mode == GridCacheMode.LOCAL || mode == GridCacheMode.REPLICATED ||
-                    (tx != null && (tx.dht() || tx.colocated()) && tx.local()))
+                    (tx != null && tx.local() && !isNear()))
                     cctx.continuousQueries().onEntryUpdate(this, key, null, null, old, oldBytes);
 
                 cctx.dataStructures().onEntryUpdated(key, true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/grid/kernal/managers/deployment/GridDeploymentMessageCountSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/deployment/GridDeploymentMessageCountSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/deployment/GridDeploymentMessageCountSelfTest.java
index e16e19b..51a0e3e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/deployment/GridDeploymentMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/deployment/GridDeploymentMessageCountSelfTest.java
@@ -114,7 +114,7 @@ public class GridDeploymentMessageCountSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testCacheValueDeploymentOnPut() throws Exception { // TODO GG-9141
+    public void testCacheValueDeploymentOnPut() throws Exception {
         ClassLoader ldr = getExternalClassLoader();
 
         Class valCls = ldr.loadClass(TEST_VALUE);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
index a57799b..e1f5c5a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
@@ -64,7 +64,7 @@ public abstract class GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest extend
     /**
      * @throws Exception If failed.
      */
-    public void _testTransform() throws Exception { // TODO GG-9141
+    public void testTransform() throws Exception {
         testTransform(keyForNode(0));
 
         if (gridCount() > 1)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
index 0489e36..e120287 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
@@ -11,7 +11,6 @@ package org.gridgain.grid.kernal.processors.cache.eviction;
 
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.eviction.fifo.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
index 1ca5003..5c798fe 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
@@ -202,7 +202,7 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testGroupLock() throws Exception { // TODO GG-9141
+    public void testGroupLock() throws Exception {
         plc = new GridCacheFifoEvictionPolicy<>(100);
 
         try {
@@ -248,7 +248,7 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testPartitionGroupLock() throws Exception { // TODO GG-9141
+    public void testPartitionGroupLock() throws Exception {
         plc = new GridCacheFifoEvictionPolicy<>(100);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
index 2465b1d..3a9901e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
@@ -81,7 +81,7 @@ public class GridCacheLruNearEvictionPolicySelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception If failed.
      */
-    public void _testTransactionalNearEvictionMaxSize() throws Exception { // TODO GG-9141
+    public void testTransactionalNearEvictionMaxSize() throws Exception {
         atomicityMode = TRANSACTIONAL;
 
         checkNearEvictionMaxSize();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 2984da1..f81cb45 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -15,7 +15,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.cache.query.GridCacheContinuousQueryEntry;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/core/src/test/java/org/gridgain/testsuites/GridCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/GridCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/GridCacheEvictionSelfTestSuite.java
index e5288b9..8b6ab21 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/GridCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/GridCacheEvictionSelfTestSuite.java
@@ -39,7 +39,7 @@ public class GridCacheEvictionSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheConcurrentEvictionsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheConcurrentEvictionConsistencySelfTest.class));
         suite.addTest(new TestSuite(GridCacheEvictionTouchSelfTest.class));
-//        suite.addTest(new TestSuite(GridCacheDistributedEvictionsSelfTest.class)); // TODO GG-9141
+        suite.addTest(new TestSuite(GridCacheDistributedEvictionsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheEvictionLockUnlockSelfTest.class));
         suite.addTest(new TestSuite(GridCacheBatchEvictUnswapSelfTest.class));
         suite.addTest(new TestSuite(GridCachePreloadingEvictionsSelfTest.class));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee2df1d8/modules/indexing/src/test/java/org/gridgain/testsuites/GridCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/testsuites/GridCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/gridgain/testsuites/GridCacheQuerySelfTestSuite.java
index bd430e3..a939fdf 100644
--- a/modules/indexing/src/test/java/org/gridgain/testsuites/GridCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/gridgain/testsuites/GridCacheQuerySelfTestSuite.java
@@ -38,8 +38,8 @@ public class GridCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedQuerySelfTest.class);
         suite.addTestSuite(GridCacheReplicatedQueryP2PDisabledSelfTest.class);
         suite.addTestSuite(GridCachePartitionedQuerySelfTest.class);
-//        suite.addTestSuite(GridCacheAtomicQuerySelfTest.class); // TODO GG-9141
-//        suite.addTestSuite(GridCacheAtomicNearEnabledQuerySelfTest.class); // TODO GG-9141
+        suite.addTestSuite(GridCacheAtomicQuerySelfTest.class);
+        suite.addTestSuite(GridCacheAtomicNearEnabledQuerySelfTest.class);
         suite.addTestSuite(GridCachePartitionedQueryP2PDisabledSelfTest.class);
         suite.addTestSuite(GridCachePartitionedQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(GridCacheQueryIndexSelfTest.class);
@@ -71,7 +71,7 @@ public class GridCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheContinuousQueryReplicatedP2PDisabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionedSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
-//        suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class); // TODO GG-9141
+        suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);


[2/3] incubator-ignite git commit: GG-9141 - Fixed group lock transactions.

Posted by ag...@apache.org.
GG-9141 - Fixed group lock transactions.


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

Branch: refs/heads/ignite-1
Commit: 59d9cd6168bdbd8721f4d9684344f57ee5c792d9
Parents: f60670a
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Dec 12 18:48:34 2014 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Dec 12 18:48:34 2014 -0800

----------------------------------------------------------------------
 .../processors/dataload/GridDataLoaderProcessorSelfTest.java    | 5 +----
 .../org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java   | 4 ++--
 2 files changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/59d9cd61/modules/core/src/test/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessorSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessorSelfTest.java
index 2f4d162..658e44b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessorSelfTest.java
@@ -14,7 +14,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.eviction.fifo.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -303,11 +302,9 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * TODO GG-4121
-     *
      * @throws Exception If failed.
      */
-    public void _testReplicatedMultiThreadedGroupLock() throws Exception {
+    public void testReplicatedMultiThreadedGroupLock() throws Exception {
         mode = REPLICATED;
         useGrpLock = true;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/59d9cd61/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
index 3bed59a..c2fe016 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
@@ -61,7 +61,7 @@ public class GridDataGridTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheBalancingStoreSelfTest.class);
         suite.addTestSuite(GridCacheAffinityApiSelfTest.class);
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
-        //suite.addTestSuite(GridDataLoaderProcessorSelfTest.class); TODO GG-9141
+        suite.addTestSuite(GridDataLoaderProcessorSelfTest.class);
         suite.addTestSuite(GridDataLoaderImplSelfTest.class);
         suite.addTestSuite(GridCacheEntryMemorySizeSelfTest.class);
         suite.addTestSuite(GridCacheClearAllSelfTest.class);
@@ -284,7 +284,7 @@ public class GridDataGridTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReloadSelfTest.class);
 
         // Group locking.
-//        suite.addTest(GridCacheGroupLockSelfTestSuite.suite());
+        suite.addTest(GridCacheGroupLockSelfTestSuite.suite());
 
         // Full API.
         suite.addTest(GridCacheFullApiSelfTestSuite.suite());