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

ignite git commit: ignite-3478 Tests restructured

Repository: ignite
Updated Branches:
  refs/heads/ignite-3478 b69f62eb6 -> b7dce2b56


ignite-3478 Tests restructured


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

Branch: refs/heads/ignite-3478
Commit: b7dce2b56a884976a8812e097dfdf11b6e8df4e4
Parents: b69f62e
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 16 15:33:51 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 16 15:49:20 2017 +0300

----------------------------------------------------------------------
 .../cache/mvcc/CacheMvccAbstractTest.java       | 75 +++++++++++++++++---
 .../cache/mvcc/CacheMvccTransactionsTest.java   | 19 +++--
 .../cache/mvcc/CacheMvccSqlQueriesTest.java     | 39 ++++++++++
 3 files changed, 118 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b7dce2b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
index f9ac96f..999144f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccAbstractTest.java
@@ -38,6 +38,9 @@ import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -64,6 +67,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -160,6 +164,7 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
      * @param clients Number of client nodes.
      * @param cacheBackups Number of cache backups.
      * @param cacheParts Number of cache partitions.
+     * @param cfgC Optional closure applied to cache configuration.
      * @param withRmvs If {@code true} then in addition to puts tests also executes removes.
      * @param readMode Read mode.
      * @throws Exception If failed.
@@ -169,6 +174,7 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
         final int clients,
         int cacheBackups,
         int cacheParts,
+        @Nullable IgniteInClosure<CacheConfiguration> cfgC,
         final boolean withRmvs,
         final ReadMode readMode
     )
@@ -335,17 +341,60 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
                         Map<Integer, MvccTestAccount> accounts;
 
                         try {
-                            if (readMode == ReadMode.SCAN) {
-                                accounts = new HashMap<>();
+                            switch (readMode) {
+                                case GET_ALL: {
+                                    accounts = cache.cache.getAll(keys);
 
-                                for (IgniteCache.Entry<Integer, MvccTestAccount> e : cache.cache) {
-                                    MvccTestAccount old = accounts.put(e.getKey(), e.getValue());
+                                    break;
+                                }
+
+                                case SCAN: {
+                                    accounts = new HashMap<>();
 
-                                    assertNull(old);
+                                    for (IgniteCache.Entry<Integer, MvccTestAccount> e : cache.cache) {
+                                        MvccTestAccount old = accounts.put(e.getKey(), e.getValue());
+
+                                        assertNull(old);
+                                    }
+
+                                    break;
+                                }
+
+                                case SQL_ALL: {
+                                    accounts = new HashMap<>();
+
+                                    if (rnd.nextBoolean()) {
+                                        SqlQuery<Integer, MvccTestAccount> qry =
+                                            new SqlQuery<>(MvccTestAccount.class, "_key >= 0");
+
+                                        for (IgniteCache.Entry<Integer, MvccTestAccount> e : cache.cache.query(qry)) {
+                                            MvccTestAccount old = accounts.put(e.getKey(), e.getValue());
+
+                                            assertNull(old);
+                                        }
+                                    }
+                                    else {
+                                        SqlFieldsQuery qry = new SqlFieldsQuery("select _key, val from MvccTestAccount");
+
+                                        for (List<?> row : cache.cache.query(qry)) {
+                                            Integer id = (Integer)row.get(0);
+                                            Integer val = (Integer)row.get(0);
+
+                                            MvccTestAccount old = accounts.put(id, new MvccTestAccount(val, 1));
+
+                                            assertNull(old);
+                                        }
+                                    }
+
+                                    break;
+                                }
+
+                                default: {
+                                    fail();
+
+                                    return;
                                 }
                             }
-                            else
-                                accounts = cache.cache.getAll(keys);
                         }
                         finally {
                             cache.readUnlock();
@@ -415,6 +464,7 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
             writers,
             readers,
             DFLT_TEST_TIME,
+            cfgC,
             init,
             writer,
             reader);
@@ -427,6 +477,7 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
      * @param cacheBackups Number of cache backups.
      * @param cacheParts Number of cache partitions.
      * @param time Test time.
+     * @param cfgC Optional closure applied to cache configuration.
      * @param writers Number of writers.
      * @param readers Number of readers.
      * @param init Optional init closure.
@@ -443,6 +494,7 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
         final int writers,
         final int readers,
         final long time,
+        @Nullable IgniteInClosure<CacheConfiguration> cfgC,
         IgniteInClosure<IgniteCache<Object, Object>> init,
         final GridInClosure3<Integer, List<TestCache>, AtomicBoolean> writer,
         final GridInClosure3<Integer, List<TestCache>, AtomicBoolean> reader) throws Exception {
@@ -467,6 +519,9 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
         if (restartMode == RestartMode.RESTART_CRD)
             ccfg.setNodeFilter(new CoordinatorNodeFilter());
 
+        if (cfgC != null)
+            cfgC.apply(ccfg);
+
         IgniteCache<Object, Object> cache = srv0.createCache(ccfg);
 
         int crdIdx = srvs + clients;
@@ -738,6 +793,7 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
      */
     static class MvccTestAccount {
         /** */
+        @QuerySqlField(index = false)
         final int val;
 
         /** */
@@ -768,7 +824,10 @@ public abstract class CacheMvccAbstractTest extends GridCommonAbstractTest {
         GET_ALL,
 
         /** */
-        SCAN
+        SCAN,
+
+        /** */
+        SQL_ALL
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7dce2b5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
index 3bfbb93..dbe4ce5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
@@ -1475,6 +1475,7 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
             readers,
             DFLT_TEST_TIME,
             null,
+            null,
             writer,
             reader);
 
@@ -1486,49 +1487,49 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
      * @throws Exception If failed.
      */
     public void testAccountsTxGetAll_SingleNode() throws Exception {
-        accountsTxReadAll(1, 0, 0, 64, false, ReadMode.GET_ALL);
+        accountsTxReadAll(1, 0, 0, 64, null, false, ReadMode.GET_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountsTxGetAll_SingleNode_SinglePartition() throws Exception {
-        accountsTxReadAll(1, 0, 0, 1, false, ReadMode.GET_ALL);
+        accountsTxReadAll(1, 0, 0, 1, null, false, ReadMode.GET_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountsTxGetAll_WithRemoves_SingleNode_SinglePartition() throws Exception {
-        accountsTxReadAll(1, 0, 0, 1, true, ReadMode.GET_ALL);
+        accountsTxReadAll(1, 0, 0, 1, null, true, ReadMode.GET_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountsTxGetAll_ClientServer_Backups0() throws Exception {
-        accountsTxReadAll(4, 2, 0, 64, false, ReadMode.GET_ALL);
+        accountsTxReadAll(4, 2, 0, 64, null, false, ReadMode.GET_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountsTxGetAll_ClientServer_Backups1() throws Exception {
-        accountsTxReadAll(4, 2, 1, 64, false, ReadMode.GET_ALL);
+        accountsTxReadAll(4, 2, 1, 64, null, false, ReadMode.GET_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountsTxGetAll_ClientServer_Backups2() throws Exception {
-        accountsTxReadAll(4, 2, 2, 64, false, ReadMode.GET_ALL);
+        accountsTxReadAll(4, 2, 2, 64, null, false, ReadMode.GET_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAccountsTxScan_SingleNode_SinglePartition() throws Exception {
-        accountsTxReadAll(1, 0, 0, 1, false, ReadMode.SCAN);
+        accountsTxReadAll(1, 0, 0, 1, null, false, ReadMode.SCAN);
     }
 
     /**
@@ -1796,6 +1797,7 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
             writers,
             readers,
             DFLT_TEST_TIME,
+            null,
             init,
             writer,
             reader);
@@ -1947,6 +1949,7 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
             readers,
             time,
             null,
+            null,
             writer,
             reader);
     }
@@ -2085,6 +2088,7 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
             readers,
             DFLT_TEST_TIME,
             null,
+            null,
             writer,
             reader);
 
@@ -3449,6 +3453,7 @@ public class CacheMvccTransactionsTest extends CacheMvccAbstractTest {
             writers,
             readers,
             time,
+            null,
             init,
             writer,
             reader);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7dce2b5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesTest.java
new file mode 100644
index 0000000..7ba1b32
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccSqlQueriesTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.mvcc;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.lang.IgniteInClosure;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class CacheMvccSqlQueriesTest extends CacheMvccAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAccountsTxSql_SingleNode_SinglePartition() throws Exception {
+        accountsTxReadAll(1, 0, 0, 1, new IgniteInClosure<CacheConfiguration>() {
+            @Override public void apply(CacheConfiguration ccfg) {
+                ccfg.setIndexedTypes(Integer.class, MvccTestAccount.class);
+            }
+        }, false, ReadMode.SQL_ALL);
+    }
+
+}