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 2016/12/20 13:04:01 UTC

[1/6] ignite git commit: ignite-4371 Neat TX finish request processing may fall into sync wait of dht finish response - enabled striped pool usage for GridDistributedTxFinishResponse - convert exceptions from CacheStoreSessionListener.onSessionStart into

Repository: ignite
Updated Branches:
  refs/heads/ignite-4450 c59f1f29f -> 4f76b75a0


ignite-4371 Neat TX finish request processing may fall into sync wait of dht finish response
- enabled striped pool usage for GridDistributedTxFinishResponse
- convert exceptions from CacheStoreSessionListener.onSessionStart into IgniteCheckedException
- should not use IgniteTxLocalAdapter.commitErr for rollback exceptions (otherwise cyclic reference in exception->cause is possible, this leads to infinite loop in 'X.hasCause')


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

Branch: refs/heads/ignite-4450
Commit: cf7b6a0e9bc6fb48d6268bfa40e1aedc9f8c9252
Parents: fcf5d9a
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 14:49:25 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 14:49:25 2016 +0300

----------------------------------------------------------------------
 .../GridDistributedTxFinishResponse.java        |  6 ----
 .../cache/store/CacheStoreManager.java          |  4 ++-
 .../store/GridCacheStoreManagerAdapter.java     | 34 ++++++++++++--------
 .../cache/store/GridCacheWriteBehindStore.java  |  6 ++--
 .../transactions/IgniteTxLocalAdapter.java      | 27 ++++++----------
 5 files changed, 36 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index c5cf332..109d665 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -86,12 +86,6 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public int partition() {
-        // TODO https://issues.apache.org/jira/browse/IGNITE-4371
-        return Integer.MIN_VALUE;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index 67c9334..8d6b63d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -171,8 +171,10 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
 
     /**
      * End session initiated by write-behind store.
+     *
+     * @throws IgniteCheckedException If failed.
      */
-    public void writeBehindSessionInit();
+    public void writeBehindSessionInit() throws IgniteCheckedException;
 
     /**
      * End session initiated by write-behind store.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 024375e..8b93afa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -288,7 +288,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public Object load(@Nullable IgniteInternalTx tx, KeyCacheObject key)
+    @Override @Nullable public final Object load(@Nullable IgniteInternalTx tx, KeyCacheObject key)
         throws IgniteCheckedException {
         return loadFromStore(tx, key, true);
     }
@@ -377,7 +377,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void localStoreLoadAll(@Nullable IgniteInternalTx tx, Collection keys, GridInClosure3 vis)
+    @Override public final void localStoreLoadAll(@Nullable IgniteInternalTx tx, Collection keys, GridInClosure3 vis)
         throws IgniteCheckedException {
         assert store != null;
         assert locStore;
@@ -386,7 +386,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loadAll(@Nullable IgniteInternalTx tx, Collection keys, IgniteBiInClosure vis)
+    @Override public final boolean loadAll(@Nullable IgniteInternalTx tx, Collection keys, IgniteBiInClosure vis)
         throws IgniteCheckedException {
         if (store != null) {
             loadAllFromStore(tx, keys, vis, null);
@@ -499,7 +499,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loadCache(final GridInClosure3 vis, Object[] args) throws IgniteCheckedException {
+    @Override public final boolean loadCache(final GridInClosure3 vis, Object[] args) throws IgniteCheckedException {
         if (store != null) {
             if (log.isDebugEnabled())
                 log.debug("Loading all values from store.");
@@ -554,7 +554,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
+    @Override public final boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
         throws IgniteCheckedException {
         if (store != null) {
             // Never persist internal keys.
@@ -599,7 +599,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
+    @Override public final boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
         if (F.isEmpty(map))
             return true;
 
@@ -658,7 +658,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
+    @Override public final boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
         if (store != null) {
             // Never remove internal key from store as it is never persisted.
             if (key instanceof GridCacheInternal)
@@ -701,7 +701,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
+    @Override public final boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
@@ -763,7 +763,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
+    @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
         assert store != null;
 
         sessionInit0(tx);
@@ -807,7 +807,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void writeBehindSessionInit() {
+    @Override public void writeBehindSessionInit() throws IgniteCheckedException {
         sessionInit0(null);
     }
 
@@ -818,8 +818,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
     /**
      * @param tx Current transaction.
+     * @throws IgniteCheckedException If failed.
      */
-    private void sessionInit0(@Nullable IgniteInternalTx tx) {
+    private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedException {
         assert sesHolder != null;
 
         SessionData ses;
@@ -841,9 +842,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         sesHolder.set(ses);
 
-        if (sesLsnrs != null && !ses.started(this)) {
-            for (CacheStoreSessionListener lsnr : sesLsnrs)
-                lsnr.onSessionStart(locSes);
+        try {
+            if (sesLsnrs != null && !ses.started(this)) {
+                for (CacheStoreSessionListener lsnr : sesLsnrs)
+                    lsnr.onSessionStart(locSes);
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to start store session: " + e, e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
index 858d9a7..f5c0c12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
@@ -673,10 +673,10 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
         Map<K, Entry<? extends K, ? extends  V>> vals,
         boolean initSes) {
 
-        if (initSes && storeMgr != null)
-            storeMgr.writeBehindSessionInit();
-
         try {
+            if (initSes && storeMgr != null)
+                storeMgr.writeBehindSessionInit();
+
             boolean threwEx = true;
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cf7b6a0e/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 0327247..e1479f9 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
@@ -1063,7 +1063,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
             setRollbackOnly();
 
             throw new IgniteCheckedException("Invalid transaction state for rollback [state=" + state +
-                ", tx=" + this + ']', commitErr);
+                ", tx=" + this + ']');
         }
 
         if (near()) {
@@ -1074,28 +1074,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         }
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {
-            try {
-                cctx.tm().rollbackTx(this);
+            cctx.tm().rollbackTx(this);
 
-                if (!internal()) {
-                    Collection<CacheStoreManager> stores = txState.stores(cctx);
+            if (!internal()) {
+                Collection<CacheStoreManager> stores = txState.stores(cctx);
 
-                    if (stores != null && !stores.isEmpty()) {
-                        assert isWriteToStoreFromDhtValid(stores) :
-                            "isWriteToStoreFromDht can't be different within one transaction";
+                if (stores != null && !stores.isEmpty()) {
+                    assert isWriteToStoreFromDhtValid(stores) :
+                        "isWriteToStoreFromDht can't be different within one transaction";
 
-                        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+                    boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
 
-                        if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
-                            sessionEnd(stores, false);
-                    }
+                    if (!stores.isEmpty() && (near() || isWriteToStoreFromDht))
+                        sessionEnd(stores, false);
                 }
             }
-            catch (Error | IgniteCheckedException | RuntimeException e) {
-                U.addLastCause(e, commitErr, log);
-
-                throw e;
-            }
         }
     }
 


[3/6] ignite git commit: Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

Posted by sb...@apache.org.
Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.


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

Branch: refs/heads/ignite-4450
Commit: 126ab60fe6fa0f47e19a26dafecc7feb7c57b60b
Parents: c0f1a21
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:15:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:15:04 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/126ab60f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 50869be..74d1553 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2951,6 +2951,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /**
      * @param key Key.
+     * @param filter Filter.
      * @return {@code True} if entry was removed.
      * @throws IgniteCheckedException If failed.
      */
@@ -2962,7 +2963,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     Collections.singletonList(key),
                     /*retval*/false,
                     filter,
-                    /*singleRmv*/true).get().success();
+                    /*singleRmv*/filter == null).get().success();
             }
 
             @Override public String toString() {


[6/6] ignite git commit: ignite-4450 added test

Posted by sb...@apache.org.
ignite-4450 added test


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

Branch: refs/heads/ignite-4450
Commit: 4f76b75a020c7f5d462f1b66d45c61b568ac44dd
Parents: eda71c0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 16:03:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 16:03:51 2016 +0300

----------------------------------------------------------------------
 .../CacheLockReleaseNodeLeaveTest.java          | 101 +++++++++++++++++++
 1 file changed, 101 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4f76b75a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
index e84fd3f..f0f18c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
@@ -20,9 +20,13 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import java.util.concurrent.Callable;
 import java.util.concurrent.locks.Lock;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -158,4 +162,101 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest {
 
         fut2.get(5, SECONDS);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockRelease2() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        Lock lock = ignite1.cache(null).lock("key");
+        lock.lock();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        final AffinityTopologyVersion topVer = new AffinityTopologyVersion(2, 0);
+
+        // Wait when affinity change exchange start.
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                AffinityTopologyVersion topVer0 =
+                    ((IgniteKernal)ignite0).context().cache().context().exchange().topologyVersion();
+
+                return topVer.compareTo(topVer0) < 0;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        lock = ignite2.cache(null).lock("key");
+        lock.lock();
+        lock.unlock();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxLockRelease2() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        IgniteCache cache = ignite1.cache(null);
+        ignite1.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+        cache.get(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        final AffinityTopologyVersion topVer = new AffinityTopologyVersion(2, 0);
+
+        // Wait when affinity change exchange start.
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                AffinityTopologyVersion topVer0 =
+                    ((IgniteKernal)ignite0).context().cache().context().exchange().topologyVersion();
+
+                return topVer.compareTo(topVer0) < 0;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        cache = ignite2.cache(null);
+
+        try (Transaction tx = ignite2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.get(1);
+
+            tx.commit();
+        }
+    }
 }


[5/6] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-4450

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-4450


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

Branch: refs/heads/ignite-4450
Commit: eda71c0c008603c392f60ec445bdcb988cfcc369
Parents: c59f1f2 3bae858
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:40:24 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:40:24 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  5 ++-
 .../GridDistributedTxFinishResponse.java        |  6 ----
 .../cache/store/CacheStoreManager.java          |  4 ++-
 .../store/GridCacheStoreManagerAdapter.java     | 34 ++++++++++++--------
 .../cache/store/GridCacheWriteBehindStore.java  |  6 ++--
 .../transactions/IgniteTxLocalAdapter.java      | 27 ++++++----------
 .../Cache/Query/CacheLinqTest.cs                | 24 ++++++++++----
 .../Impl/CacheQueryExpressionVisitor.cs         | 12 ++-----
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 34 ++++++++++++++++++--
 9 files changed, 92 insertions(+), 60 deletions(-)
----------------------------------------------------------------------



[4/6] ignite git commit: IGNITE-4359 .NET: Support DateTime properties in LINQ

Posted by sb...@apache.org.
IGNITE-4359 .NET: Support DateTime properties in LINQ


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

Branch: refs/heads/ignite-4450
Commit: 3bae858986f78dbb288cf39d970b3f163c3da247
Parents: 126ab60
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Dec 20 15:32:39 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 15:32:39 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 24 ++++++++++----
 .../Impl/CacheQueryExpressionVisitor.cs         | 12 ++-----
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 34 ++++++++++++++++++--
 3 files changed, 52 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index 1ac7fa7..46979ef 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -63,7 +63,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         private bool _runDbConsole;
 
         /** */
-        private static readonly DateTime StartDateTime = new DateTime(2000, 1, 1, 0, 0, 0, DateTimeKind.Utc);
+        private static readonly DateTime StartDateTime = new DateTime(2000, 5, 17, 15, 4, 5, DateTimeKind.Utc);
 
         /// <summary>
         /// Fixture set up.
@@ -111,7 +111,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             roleCache[new RoleKey(1, 101)] = new Role {Name = "Role_1", Date = StartDateTime};
             roleCache[new RoleKey(2, 102)] = new Role {Name = "Role_2", Date = StartDateTime.AddYears(1)};
-            roleCache[new RoleKey(3, 103)] = new Role {Name = null, Date = StartDateTime.AddYears(2)};
+            roleCache[new RoleKey(3, 103)] = new Role {Name = null, Date = StartDateTime.AddHours(5432)};
         }
 
         /// <summary>
@@ -123,7 +123,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             {
                 BinaryConfiguration = new BinaryConfiguration(typeof(Person),
                     typeof(Organization), typeof(Address), typeof(Role), typeof(RoleKey), typeof(Numerics)),
-                GridName = gridName
+                GridName = gridName,
+                JvmOptions = { "-Duser.timezone=UTC" }
             };
         }
 
@@ -735,7 +736,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             // Test retrieval
             var dates = roles.OrderBy(x => x.Value.Date).Select(x => x.Value.Date);
-            var expDates = new[] {StartDateTime, StartDateTime.AddYears(1), StartDateTime.AddYears(2)};
+            var expDates = GetRoleCache().Select(x => x.Value.Date).OrderBy(x => x).ToArray();
             Assert.AreEqual(expDates, dates.ToArray());
 
             // Filtering
@@ -748,10 +749,21 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 join person in persons on role.Value.Date equals person.Value.Birthday
                 select person;
 
-            Assert.AreEqual(RoleCount, join.Count());
+            Assert.AreEqual(2, join.Count());
 
             // Functions
-            Assert.AreEqual("01 01 2000 00:00:00", dates.Select(x => x.ToString("DD MM YYYY HH:mm:ss")).First());
+            var strings = dates.Select(x => x.ToString("dd MM YYYY HH:mm:ss")).ToArray();
+            Assert.AreEqual(new[] {"17 05 2000 15:04:05", "29 12 2000 23:04:05", "17 05 2001 15:04:05"}, strings);
+
+            // Properties
+            Assert.AreEqual(new[] {2000, 2000, 2001}, dates.Select(x => x.Year).ToArray());
+            Assert.AreEqual(new[] {5, 12, 5}, dates.Select(x => x.Month).ToArray());
+            Assert.AreEqual(new[] {17, 29, 17}, dates.Select(x => x.Day).ToArray());
+            Assert.AreEqual(expDates.Select(x => x.DayOfYear).ToArray(), dates.Select(x => x.DayOfYear).ToArray());
+            Assert.AreEqual(expDates.Select(x => x.DayOfWeek).ToArray(), dates.Select(x => x.DayOfWeek).ToArray());
+            Assert.AreEqual(new[] {15, 23, 15}, dates.Select(x => x.Hour).ToArray());
+            Assert.AreEqual(new[] { 4, 4, 4 }, dates.Select(x => x.Minute).ToArray());
+            Assert.AreEqual(new[] { 5, 5, 5 }, dates.Select(x => x.Second).ToArray());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
index 8231053..1f9da1c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -276,17 +276,9 @@ namespace Apache.Ignite.Linq.Impl
         {
             // Field hierarchy is flattened (Person.Address.Street is just Street), append as is, do not call Visit.
 
-            // Special case: string.Length
-            if (expression.Member == MethodVisitor.StringLength)
-            {
-                ResultBuilder.Append("length(");
-
-                VisitMember((MemberExpression) expression.Expression);
-
-                ResultBuilder.Append(")");
-
+            // Property call (string.Length, DateTime.Month, etc).
+            if (MethodVisitor.VisitPropertyCall(expression, this))
                 return expression;
-            }
 
             // Special case: grouping
             if (VisitGroupByMember(expression.Expression))

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index 3c8acc7..e83c448 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -29,8 +29,19 @@ namespace Apache.Ignite.Linq.Impl
     /// </summary>
     internal static class MethodVisitor
     {
-        /// <summary> The string length method. </summary>
-        public static readonly MemberInfo StringLength = typeof (string).GetProperty("Length");
+        /// <summary> Property visitors. </summary>
+        private static readonly Dictionary<MemberInfo, string> Properties = new Dictionary<MemberInfo, string>
+        {
+            {typeof(string).GetProperty("Length"), "length"},
+            {typeof(DateTime).GetProperty("Year"), "year"},
+            {typeof(DateTime).GetProperty("Month"), "month"},
+            {typeof(DateTime).GetProperty("Day"), "day_of_month"},
+            {typeof(DateTime).GetProperty("DayOfYear"), "day_of_year"},
+            {typeof(DateTime).GetProperty("DayOfWeek"), "-1 + day_of_week"},
+            {typeof(DateTime).GetProperty("Hour"), "hour"},
+            {typeof(DateTime).GetProperty("Minute"), "minute"},
+            {typeof(DateTime).GetProperty("Second"), "second"}
+        };
 
         /// <summary> Method visit delegate. </summary>
         private delegate void VisitMethodDelegate(MethodCallExpression expression, CacheQueryExpressionVisitor visitor);
@@ -103,6 +114,25 @@ namespace Apache.Ignite.Linq.Impl
         }.ToDictionary(x => x.Key, x => x.Value);
 
         /// <summary>
+        /// Visits the property call expression.
+        /// </summary>
+        public static bool VisitPropertyCall(MemberExpression expression, CacheQueryExpressionVisitor visitor)
+        {
+            string funcName;
+
+            if (!Properties.TryGetValue(expression.Member, out funcName))
+                return false;
+
+            visitor.ResultBuilder.Append(funcName).Append('(');
+
+            visitor.Visit(expression.Expression);
+
+            visitor.ResultBuilder.Append(')');
+
+            return true;
+        }
+
+        /// <summary>
         /// Visits the method call expression.
         /// </summary>
         public static void VisitMethodCall(MethodCallExpression expression, CacheQueryExpressionVisitor visitor)


[2/6] ignite git commit: Null argument check for cache.putAll.

Posted by sb...@apache.org.
Null argument check for cache.putAll.


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

Branch: refs/heads/ignite-4450
Commit: c0f1a219e0e423cb65f25a3968318495d29ab86f
Parents: cf7b6a0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 12:35:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 12:35:51 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f1a219/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 5707b49..50869be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2664,6 +2664,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        A.notNull(m, "map");
+
         if (F.isEmpty(m))
             return;