You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2016/02/27 05:28:58 UTC

[2/2] ignite git commit: IGNITE-2455 - Create missing caches in cache 'schema not found' exception is thrown

IGNITE-2455 - Create missing caches in cache 'schema not found' exception is thrown


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

Branch: refs/heads/master
Commit: affe6e746652e9c76e527a279b7db51f0d55a70c
Parents: e970d56
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Fri Feb 26 20:28:42 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Feb 26 20:28:42 2016 -0800

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  12 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 104 +++++++++++--------
 .../cache/CacheQueryNewClientSelfTest.java      |  80 ++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   6 +-
 4 files changed, 158 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/affe6e74/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index de85d03..a8f205b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -3309,6 +3309,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Starts client caches that do not exist yet.
+     *
+     * @throws IgniteCheckedException In case of error.
+     */
+    public void createMissingCaches() throws IgniteCheckedException {
+        for (String cacheName : registeredCaches.keySet()) {
+            if (!CU.isSystemCache(cacheName) && !caches.containsKey(cacheName))
+                dynamicStartCache(null, cacheName, null, false, true, true).get();
+        }
+    }
+
+    /**
      * Registers MBean for cache components.
      *
      * @param o Cache component.

http://git-wip-us.apache.org/repos/asf/ignite/blob/affe6e74/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index cae1a9f..a0e9c58 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -17,6 +17,42 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import javax.cache.Cache;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -81,6 +117,7 @@ import org.apache.ignite.spi.IgniteSpiCloseableIterator;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.CommandInterface;
+import org.h2.constant.ErrorCode;
 import org.h2.constant.SysProperties;
 import org.h2.index.Index;
 import org.h2.index.SpatialIndex;
@@ -115,43 +152,6 @@ import org.h2.value.ValueUuid;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import javax.cache.Cache;
-import javax.cache.CacheException;
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE;
 import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
@@ -1035,13 +1035,31 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         else {
             PreparedStatement stmt;
 
-            try {
-                // Do not cache this statement because the whole two step query object will be cached later on.
-                stmt = prepareStatement(c, sqlQry, false);
-            }
-            catch (SQLException e) {
-                throw new CacheException("Failed to parse query: " + sqlQry, e);
+            boolean cachesCreated = false;
+
+            while (true) {
+                try {
+                    // Do not cache this statement because the whole two step query object will be cached later on.
+                    stmt = prepareStatement(c, sqlQry, false);
+
+                    break;
+                }
+                catch (SQLException e) {
+                    if (!cachesCreated && e.getErrorCode() == ErrorCode.SCHEMA_NOT_FOUND_1) {
+                        try {
+                            ctx.cache().createMissingCaches();
+                        }
+                        catch (IgniteCheckedException e1) {
+                            throw new CacheException("Failed to create missing caches.", e);
+                        }
+
+                        cachesCreated = true;
+                    }
+                    else
+                        throw new CacheException("Failed to parse query: " + sqlQry, e);
+                }
             }
+
             try {
                 try {
                     bindParameters(stmt, F.asList(qry.getArgs()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/affe6e74/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
new file mode 100644
index 0000000..020dddd
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for the case when client is started after the cache is already created.
+ */
+public class CacheQueryNewClientSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryFromNewClient() throws Exception {
+        Ignite server = startGrid("server");
+
+        IgniteCache<Integer, Integer> cache1 = server.createCache(new CacheConfiguration<Integer, Integer>().
+            setName("cache1").setIndexedTypes(Integer.class, Integer.class));
+        IgniteCache<Integer, Integer> cache2 = server.createCache(new CacheConfiguration<Integer, Integer>().
+            setName("cache2").setIndexedTypes(Integer.class, Integer.class));
+
+        for (int i = 0; i < 10; i++) {
+            cache1.put(i, i);
+            cache2.put(i, i);
+        }
+
+        Ignition.setClientMode(true);
+
+        Ignite client = startGrid("client");
+
+        IgniteCache<Integer, Integer> cache = client.cache("cache1");
+
+        List<List<?>> res = cache.query(new SqlFieldsQuery(
+            "select i1._val, i2._val from Integer i1 cross join \"cache2\".Integer i2")).getAll();
+
+        assertEquals(100, res.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/affe6e74/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 457ab9b..968dbf6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLostTest;
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
@@ -252,10 +253,13 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
         suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
 
-        //Unmarshallig query test.
+        // Unmarshalling query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
         suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
 
+        // Other.
+        suite.addTestSuite(CacheQueryNewClientSelfTest.class);
+
         return suite;
     }
 }