Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C30C617D19 for ; Wed, 8 Apr 2015 13:44:58 +0000 (UTC) Received: (qmail 56030 invoked by uid 500); 8 Apr 2015 13:44:58 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 55996 invoked by uid 500); 8 Apr 2015 13:44:58 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 55987 invoked by uid 99); 8 Apr 2015 13:44:58 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 08 Apr 2015 13:44:58 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Wed, 08 Apr 2015 13:44:54 +0000 Received: (qmail 55606 invoked by uid 99); 8 Apr 2015 13:44:34 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 08 Apr 2015 13:44:34 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 4BE94E2F28; Wed, 8 Apr 2015 13:44:34 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Wed, 08 Apr 2015 13:44:36 -0000 Message-Id: <210adc1531db4743b84b12027b8c2f66@git.apache.org> In-Reply-To: <48a091ac02f34f15a62f1d5341631aaa@git.apache.org> References: <48a091ac02f34f15a62f1d5341631aaa@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [03/50] incubator-ignite git commit: # ignite-680 X-Virus-Checked: Checked by ClamAV on apache.org # ignite-680 Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/a815911d Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a815911d Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a815911d Branch: refs/heads/ignite-437-sqltests-p2 Commit: a815911d2f5f6f4fa5de9baad9dce6febbd6e835 Parents: dd944bf Author: sboikov Authored: Mon Apr 6 16:08:27 2015 +0300 Committer: sboikov Committed: Mon Apr 6 16:23:24 2015 +0300 ---------------------------------------------------------------------- .../src/main/java/org/apache/ignite/Ignite.java | 15 + .../apache/ignite/internal/IgniteKernal.java | 37 +++ .../internal/managers/GridManagerAdapter.java | 34 +- .../cache/CacheTemplateConfigurationKey.java | 53 ++++ .../processors/cache/GridCacheProcessor.java | 315 +++++++++++++++++-- ...eCacheConfigurationTemplateNotFoundTest.java | 114 +++++++ .../IgniteCacheConfigurationTemplateTest.java | 250 +++++++++++++++ .../ignite/testframework/junits/IgniteMock.java | 10 + .../ignite/testsuites/IgniteCacheTestSuite.java | 2 + .../org/apache/ignite/IgniteSpringBean.java | 14 + 10 files changed, 807 insertions(+), 37 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/main/java/org/apache/ignite/Ignite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java index 13e072b..6b49add 100644 --- a/modules/core/src/main/java/org/apache/ignite/Ignite.java +++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java @@ -225,6 +225,21 @@ public interface Ignite extends AutoCloseable { public IgniteCache getOrCreateCache(CacheConfiguration cacheCfg); /** + * Gets existing cache with the given name or creates new one using template configuration. + * + * @param cacheName Cache name. + * @return Existing or newly created cache. + */ + public IgniteCache getOrCreateCache(String cacheName); + + /** + * Adds cache configuration template. + * + * @param cacheCfg Cache configuration template. + */ + public void addCacheConfiguration(CacheConfiguration cacheCfg); + + /** * Dynamically starts new cache with the given cache configuration. *

* If local node is an affinity node, this method will return the instance of started cache. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index a623cb9..08ec78f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -2259,6 +2259,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { try { return ctx.cache().publicJCache(name); } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); + } finally { unguard(); } @@ -2406,6 +2409,40 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { } } + /** {@inheritDoc} */ + @Override public IgniteCache getOrCreateCache(String cacheName) { + guard(); + + try { + ctx.cache().dynamicStartCache(cacheName).get(); + + return ctx.cache().publicJCache(cacheName); + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); + } + finally { + unguard(); + } + } + + /** {@inheritDoc} */ + @Override public void addCacheConfiguration(CacheConfiguration cacheCfg) { + A.notNull(cacheCfg, "cacheCfg"); + + guard(); + + try { + ctx.cache().addCacheConfiguration(cacheCfg); + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); + } + finally { + unguard(); + } + } + /** * @return Public caches. */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java index 982ca86..83d257e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java @@ -398,27 +398,37 @@ public abstract class GridManagerAdapter implements GridMan } @Nullable @Override public V put(String cacheName, K key, V val, long ttl) { - if (ttl > 0) { - ExpiryPolicy plc = new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl)); + try { + if (ttl > 0) { + ExpiryPolicy plc = new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl)); - IgniteCache cache = ctx.cache().publicJCache(cacheName).withExpiryPolicy(plc); + IgniteCache cache = ctx.cache().publicJCache(cacheName).withExpiryPolicy(plc); - return cache.getAndPut(key, val); + return cache.getAndPut(key, val); + } + else + return ctx.cache().jcache(cacheName).getAndPut(key, val); + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); } - else - return ctx.cache().jcache(cacheName).getAndPut(key, val); } @Nullable @Override public V putIfAbsent(String cacheName, K key, V val, long ttl) { - if (ttl > 0) { - ExpiryPolicy plc = new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl)); + try { + if (ttl > 0) { + ExpiryPolicy plc = new TouchedExpiryPolicy(new Duration(MILLISECONDS, ttl)); - IgniteCache cache = ctx.cache().publicJCache(cacheName).withExpiryPolicy(plc); + IgniteCache cache = ctx.cache().publicJCache(cacheName).withExpiryPolicy(plc); - return cache.getAndPutIfAbsent(key, val); + return cache.getAndPutIfAbsent(key, val); + } + else + return ctx.cache().jcache(cacheName).getAndPutIfAbsent(key, val); + } + catch (IgniteCheckedException e) { + throw CU.convertToCacheException(e); } - else - return ctx.cache().jcache(cacheName).getAndPutIfAbsent(key, val); } @Nullable @Override public V remove(String cacheName, K key) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheTemplateConfigurationKey.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheTemplateConfigurationKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheTemplateConfigurationKey.java new file mode 100644 index 0000000..310fa74 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheTemplateConfigurationKey.java @@ -0,0 +1,53 @@ +/* + * 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.io.*; + +/** + * + */ +public class CacheTemplateConfigurationKey implements GridCacheInternal, Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public int hashCode() { + return getClass().getName().hashCode(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + return obj == this || (obj instanceof CacheTemplateConfigurationKey); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "CacheTemplateConfigurationKey []"; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/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 482d816..a7777de 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 @@ -60,6 +60,7 @@ import org.jetbrains.annotations.*; import javax.cache.configuration.*; import javax.cache.integration.*; +import javax.cache.processor.*; import javax.management.*; import java.io.*; import java.util.*; @@ -575,6 +576,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { cfgs[i] = cfg; // Replace original configuration value. + if (cfg.getName() != null && cfg.getName().endsWith("*")) // Skip template configurations. + continue; + String masked = maskNull(cfg.getName()); if (registeredCaches.containsKey(masked)) { @@ -766,6 +770,20 @@ public class GridCacheProcessor extends GridProcessorAdapter { } ctx.cacheObjects().onCacheProcessorStarted(); + + List templateCfgs = null; + + for (CacheConfiguration cfg : ctx.config().getCacheConfiguration()) { + if (cfg.getName() == null || cfg.getName().endsWith("*")) { + if (templateCfgs == null) + templateCfgs = new ArrayList<>(); + + templateCfgs.add(cfg); + } + } + + if (templateCfgs != null) + addCacheConfigurations(templateCfgs); } /** {@inheritDoc} */ @@ -1475,6 +1493,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { * * @param topVer Completed topology version. * @param reqs Change requests. + * @param err Error. */ @SuppressWarnings("unchecked") public void onExchangeDone( @@ -1644,6 +1663,39 @@ public class GridCacheProcessor extends GridProcessorAdapter { } /** + * Dynamically starts cache using template configuration. + * + * @param cacheName Cache name. + * @return Future that will be completed when cache is deployed. + */ + public IgniteInternalFuture dynamicStartCache(String cacheName) { + try { + if (publicJCache(cacheName, false) != null) // Cache with given name already started. + return new GridFinishedFuture<>(); + + GridCacheAdapter utilityCache = utilityCache(); + + TemplateConfigurations cfgs = + utilityCache.localPeek(new CacheTemplateConfigurationKey(), CachePeekModes.ONHEAP_ONLY, null); + + CacheConfiguration cfgTemplate = cfgs != null ? cfgs.configuration(cacheName) : null; + + if (cfgTemplate == null) + throw new IllegalArgumentException("Failed to start cache " + + "(there is no matching template configuration) : " + cacheName); + + CacheConfiguration cfg = new CacheConfiguration(cfgTemplate); + + cfg.setName(cacheName); + + return dynamicStartCache(cfg, cacheName, null, false); + } + catch (IgniteCheckedException e) { + return new GridFinishedFuture<>(e); + } + } + + /** * Dynamically starts cache. * * @param ccfg Cache configuration. @@ -2309,54 +2361,104 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @param type of keys. * @param type of values. * @return Cache instance for given name. + * @throws IgniteCheckedException If failed. + */ + public IgniteCache publicJCache(@Nullable String cacheName) throws IgniteCheckedException { + return publicJCache(cacheName, true); + } + + /** + * @param cacheName Cache name. + * @param failIfNotStarted If {@code true} throws {@link IllegalArgumentException} if cache is not started, + * otherwise returns {@code null} in this case. + * @param type of keys. + * @param type of values. + * @return Cache instance for given name. + * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - public IgniteCache publicJCache(@Nullable String cacheName) { + @Nullable private IgniteCache publicJCache(@Nullable String cacheName, boolean failIfNotStarted) + throws IgniteCheckedException + { if (log.isDebugEnabled()) log.debug("Getting public cache for name: " + cacheName); - try { - String masked = maskNull(cacheName); + String masked = maskNull(cacheName); - IgniteCache cache = (IgniteCache)jCacheProxies.get(masked); + IgniteCache cache = (IgniteCache)jCacheProxies.get(masked); - DynamicCacheDescriptor desc = registeredCaches.get(masked); + DynamicCacheDescriptor desc = registeredCaches.get(masked); - if (desc != null && !desc.cacheType().userCache()) - throw new IllegalStateException("Failed to get cache because it is a system cache: " + cacheName); + if (desc != null && !desc.cacheType().userCache()) + throw new IllegalStateException("Failed to get cache because it is a system cache: " + cacheName); - if (cache == null) { - if (desc == null || desc.cancelled()) + if (cache == null) { + if (desc == null || desc.cancelled()) { + if (failIfNotStarted) throw new IllegalArgumentException("Cache is not started: " + cacheName); - DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId()); + return null; + } - req.cacheName(cacheName); + DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId()); - req.deploymentId(desc.deploymentId()); + req.cacheName(cacheName); - CacheConfiguration cfg = new CacheConfiguration(desc.cacheConfiguration()); + req.deploymentId(desc.deploymentId()); - cfg.setNearConfiguration(null); + CacheConfiguration cfg = new CacheConfiguration(desc.cacheConfiguration()); - req.startCacheConfiguration(cfg); + cfg.setNearConfiguration(null); - req.cacheType(desc.cacheType()); + req.startCacheConfiguration(cfg); - req.clientStartOnly(true); + req.cacheType(desc.cacheType()); - F.first(initiateCacheChanges(F.asList(req))).get(); + req.clientStartOnly(true); - cache = (IgniteCache)jCacheProxies.get(masked); + F.first(initiateCacheChanges(F.asList(req))).get(); - if (cache == null) - throw new IllegalArgumentException("Cache is not started: " + cacheName); - } + cache = (IgniteCache)jCacheProxies.get(masked); - return cache; + if (cache == null && failIfNotStarted) + throw new IllegalArgumentException("Cache is not started: " + cacheName); } - catch (IgniteCheckedException e) { - throw CU.convertToCacheException(e); + + return cache; + } + + /** + * @param cacheCfg Cache configuration template. + * @throws IgniteCheckedException If failed. + */ + public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteCheckedException { + addCacheConfigurations(Collections.singleton(cacheCfg)); + } + + /** + * @param cfgs Cache configurations. + * @throws IgniteCheckedException If failed. + */ + private void addCacheConfigurations(Collection cfgs) throws IgniteCheckedException { + GridCacheAdapter utilityCache = utilityCache(); + + final int RETRY_CNT = 5; + + for (int i = 0; i < RETRY_CNT; i++) { + try { + utilityCache.invoke( + new CacheTemplateConfigurationKey(), new AddCacheTemplateProcessor(cfgs)); + + break; + } + catch (IgniteCheckedException e) { + if (i < RETRY_CNT - 1) { + if (log.isDebugEnabled()) + log.debug("Failed to save cache template configuration, will retry: " + e); + } + else + throw e; + } } } @@ -2571,6 +2673,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { /** * @param val Object to check. + * @throws IgniteCheckedException If validation failed. */ private void checkSerializable(CacheConfiguration val) throws IgniteCheckedException { if (val == null) @@ -2606,6 +2709,165 @@ public class GridCacheProcessor extends GridProcessorAdapter { /** * */ + private class AddCacheTemplateProcessor implements + CacheEntryProcessor { + /** */ + private Collection cfgs; + + /** + * @param cfgs Cache configuration templates. + */ + public AddCacheTemplateProcessor(Collection cfgs) { + assert !F.isEmpty(cfgs); + + this.cfgs = cfgs; + } + + /** {@inheritDoc} */ + @Override public Void process(MutableEntry entry, + Object... args) + { + TemplateConfigurations cfgs0 = entry.getValue(); + + if (cfgs0 == null) + cfgs0 = new TemplateConfigurations(); + + cfgs0.add(cfgs); + + entry.setValue(cfgs0); + + return null; + } + } + + /** + * + */ + private static class TemplateConfigurations implements Externalizable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + @GridToStringInclude + private List exactNameCfgs; + + /** */ + @GridToStringInclude + private List wildcardNameCfgs; + + /** + * + */ + public TemplateConfigurations() { + // No-op. + } + + /** + * @param cfgs Configurations to add. + */ + void add(Collection cfgs) { + for (CacheConfiguration cfg : cfgs) { + CacheConfiguration template = new CacheConfiguration(cfg); + + if (cfg.getName() != null && cfg.getName().endsWith("*")) { + template.setName(cfg.getName().substring(0, cfg.getName().length() - 1)); + + boolean found = false; + + if (wildcardNameCfgs != null) { + for (CacheConfiguration cfg0 : wildcardNameCfgs) { + if (F.eq(cfg0.getName(), template.getName())) { + found = true; + + break; + } + } + } + else + wildcardNameCfgs = new ArrayList<>(cfgs.size()); + + if (!found) + wildcardNameCfgs.add(template); + } + else { + boolean found = false; + + if (exactNameCfgs != null) { + for (CacheConfiguration cfg0 : exactNameCfgs) { + if (F.eq(cfg0.getName(), template.getName())) { + found = true; + + break; + } + } + } + else + exactNameCfgs = new ArrayList<>(cfgs.size()); + + if (!found) + exactNameCfgs.add(template); + } + } + + if (wildcardNameCfgs != null) { + Collections.sort(wildcardNameCfgs, new Comparator() { + @Override public int compare(CacheConfiguration cfg1, CacheConfiguration cfg2) { + Integer len1 = cfg1.getName() != null ? cfg1.getName().length() : 0; + Integer len2 = cfg2.getName() != null ? cfg2.getName().length() : 0; + + return len2.compareTo(len1); + } + }); + } + } + + /** + * @param name Cache name. + * @return Cache configuration for given name or {@code null} if template not found. + */ + @Nullable CacheConfiguration configuration(String name) { + CacheConfiguration dfltCfg = null; + + if (exactNameCfgs != null) { + for (CacheConfiguration cfg : exactNameCfgs) { + if (F.eq(name, cfg.getName())) + return cfg; + else if (cfg.getName() == null) + dfltCfg = cfg; + } + } + + if (wildcardNameCfgs != null && name != null) { + for (CacheConfiguration cfg : wildcardNameCfgs) { + if (name.startsWith(cfg.getName())) + return cfg; + } + } + + return dfltCfg; + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeCollection(out, exactNameCfgs); + U.writeCollection(out, wildcardNameCfgs); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + exactNameCfgs = U.readList(in); + wildcardNameCfgs = U.readList(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TemplateConfigurations.class, this); + } + } + + /** + * + */ @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor") private class DynamicCacheStartFuture extends GridFutureAdapter { /** Start ID. */ @@ -2620,6 +2882,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { private DynamicCacheChangeRequest req; /** + * @param cacheName Cache name. + * @param deploymentId Deployment ID. + * @param req Cache start request. */ private DynamicCacheStartFuture(String cacheName, IgniteUuid deploymentId, DynamicCacheChangeRequest req) { this.deploymentId = deploymentId; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateNotFoundTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateNotFoundTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateNotFoundTest.java new file mode 100644 index 0000000..53780da --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateNotFoundTest.java @@ -0,0 +1,114 @@ +/* + * 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 org.apache.ignite.*; +import org.apache.ignite.configuration.*; +import org.apache.ignite.spi.discovery.tcp.*; +import org.apache.ignite.spi.discovery.tcp.ipfinder.*; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; +import org.apache.ignite.testframework.*; +import org.apache.ignite.testframework.junits.common.*; + +import java.util.concurrent.*; + +/** + * + */ +public class IgniteCacheConfigurationTemplateNotFoundTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + CacheConfiguration templateCfg = new CacheConfiguration(); + + templateCfg.setName("org.apache.ignite.template*"); + templateCfg.setBackups(3); + + cfg.setCacheConfiguration(templateCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testTemplateNotFound() throws Exception { + Ignite ignite = startGrid(0); + + checkTemplateNotFound(ignite, "org.apache.ignite"); + + checkTemplateNotFound(ignite, "org.apache.ignite.templat"); + + checkTemplateNotFound(ignite, null); + + checkGetOrCreate(ignite, "org.apache.ignite.template", 3); + + CacheConfiguration templateCfg = new CacheConfiguration(); + + templateCfg.setBackups(4); + + ignite.addCacheConfiguration(templateCfg); + + checkGetOrCreate(ignite, "org.apache.ignite", 4); + + checkGetOrCreate(ignite, null, 4); + } + + /** + * @param ignite Ignite. + * @param cacheName Cache name. + */ + private void checkTemplateNotFound(final Ignite ignite, final String cacheName) { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + ignite.getOrCreateCache(cacheName); + + return null; + } + }, IllegalArgumentException.class, null); + } + + /** + * @param ignite Ignite. + * @param name Cache name. + * @param expBackups Expected number of backups. + */ + private void checkGetOrCreate(Ignite ignite, String name, int expBackups) { + IgniteCache cache = ignite.getOrCreateCache(name); + + assertNotNull(cache); + + CacheConfiguration cfg = (CacheConfiguration)cache.getConfiguration(CacheConfiguration.class); + + assertEquals(name, cfg.getName()); + assertEquals(expBackups, cfg.getBackups()); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java new file mode 100644 index 0000000..c32d54b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java @@ -0,0 +1,250 @@ +/* + * 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 org.apache.ignite.*; +import org.apache.ignite.configuration.*; +import org.apache.ignite.spi.discovery.tcp.*; +import org.apache.ignite.spi.discovery.tcp.ipfinder.*; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; +import org.apache.ignite.testframework.*; +import org.apache.ignite.testframework.junits.common.*; + +import java.util.concurrent.*; + +import static org.apache.ignite.cache.CacheAtomicityMode.*; + +/** + * + */ +public class IgniteCacheConfigurationTemplateTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String TEMPLATE1 = "org.apache.ignite*"; + + /** */ + private static final String TEMPLATE2 = "org.apache.ignite.test.*"; + + /** */ + private static final String TEMPLATE3 = "org.apache.ignite.test2.*"; + + /** */ + private boolean clientMode; + + /** */ + private boolean addTemplate; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + if (addTemplate) { + CacheConfiguration dfltCfg = new CacheConfiguration(); + + dfltCfg.setAtomicityMode(TRANSACTIONAL); + dfltCfg.setBackups(2); + + CacheConfiguration templateCfg1 = new CacheConfiguration(); + + templateCfg1.setName(TEMPLATE1); + templateCfg1.setBackups(3); + + CacheConfiguration templateCfg2 = new CacheConfiguration(); + + templateCfg2.setName(TEMPLATE2); + templateCfg2.setBackups(4); + + cfg.setCacheConfiguration(dfltCfg, templateCfg1, templateCfg2); + } + + cfg.setClientMode(clientMode); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testCreateFromTemplateConfiguration() throws Exception { + addTemplate = true; + + Ignite ignite0 = startGrid(0); + + checkNoTemplateCaches(1); + + checkGetOrCreate(ignite0, "org.apache.ignite.test.cache1", 4); + checkGetOrCreate(ignite0, "org.apache.ignite.test.cache1", 4); + + Ignite ignite1 = startGrid(1); + + checkGetOrCreate(ignite1, "org.apache.ignite.test.cache1", 4); + checkGetOrCreate(ignite1, "org.apache.ignite.test.cache1", 4); + + checkGetOrCreate(ignite1, "org.apache.ignite1", 3); + checkGetOrCreate(ignite1, "org.apache.ignite1", 3); + + checkGetOrCreate(ignite0, "org.apache.ignite1", 3); + checkGetOrCreate(ignite0, "org.apache.ignite1", 3); + + checkGetOrCreate(ignite0, "org.apache1", 2); + checkGetOrCreate(ignite1, "org.apache1", 2); + + checkNoTemplateCaches(2); + + addTemplate = false; + clientMode = true; + + Ignite ignite2 = startGrid(2); + + assertNotNull(ignite2.cache("org.apache.ignite.test.cache1")); + assertNotNull(ignite2.cache("org.apache.ignite1")); + assertNotNull(ignite2.cache("org.apache1")); + + checkGetOrCreate(ignite2, "org.apache.ignite.test.cache1", 4); + checkGetOrCreate(ignite2, "org.apache.ignite1", 3); + checkGetOrCreate(ignite2, "org.apache1", 2); + + checkGetOrCreate(ignite2, "org.apache.ignite.test.cache2", 4); + checkGetOrCreate(ignite2, "org.apache.ignite.cache2", 3); + checkGetOrCreate(ignite2, "org.apache2", 2); + + CacheConfiguration template1 = new CacheConfiguration(); + + template1.setName(TEMPLATE3); + template1.setBackups(5); + + ignite2.addCacheConfiguration(template1); + + checkGetOrCreate(ignite0, "org.apache.ignite.test2.cache1", 5); + checkGetOrCreate(ignite1, "org.apache.ignite.test2.cache1", 5); + checkGetOrCreate(ignite2, "org.apache.ignite.test2.cache1", 5); + + Ignite ignite3 = startGrid(3); + + checkGetOrCreate(ignite3, "org.apache.ignite.test2.cache1", 5); + + checkNoTemplateCaches(4); + + // Template with non-wildcard name. + CacheConfiguration template2 = new CacheConfiguration(); + + template2.setName("org.apache.ignite"); + template2.setBackups(6); + + ignite0.addCacheConfiguration(template2); + + checkGetOrCreate(ignite0, "org.apache.ignite", 6); + checkGetOrCreate(ignite1, "org.apache.ignite", 6); + checkGetOrCreate(ignite2, "org.apache.ignite", 6); + checkGetOrCreate(ignite3, "org.apache.ignite", 6); + } + + /** + * @throws Exception If failed. + */ + public void testStartClientNodeFirst() throws Exception { + addTemplate = true; + clientMode = true; + + Ignite ignite0 = startGrid(0); + + checkNoTemplateCaches(0); + + addTemplate = false; + clientMode = false; + + Ignite ignite1 = startGrid(1); + + checkGetOrCreate(ignite1, "org.apache.ignite.test.cache1", 4); + checkGetOrCreate(ignite1, "org.apache.ignite.test.cache1", 4); + + checkGetOrCreate(ignite1, "org.apache.ignite1", 3); + checkGetOrCreate(ignite1, "org.apache.ignite1", 3); + + checkGetOrCreate(ignite0, "org.apache.ignite1", 3); + checkGetOrCreate(ignite0, "org.apache.ignite1", 3); + } + + /** + * @param ignite Ignite. + * @param name Cache name. + * @param expBackups Expected number of backups. + */ + private void checkGetOrCreate(Ignite ignite, String name, int expBackups) { + IgniteCache cache = ignite.getOrCreateCache(name); + + assertNotNull(cache); + + CacheConfiguration cfg = (CacheConfiguration)cache.getConfiguration(CacheConfiguration.class); + + assertEquals(name, cfg.getName()); + assertEquals(expBackups, cfg.getBackups()); + } + + /** + * @param nodes Nodes number. + */ + private void checkNoTemplateCaches(int nodes) { + for (int i = 0; i < nodes; i++) { + final Ignite ignite = grid(i); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + ignite.cache(GridCacheUtils.UTILITY_CACHE_NAME); + + return null; + } + }, IllegalStateException.class, null); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + ignite.cache(TEMPLATE1); + + return null; + } + }, IllegalArgumentException.class, null); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + ignite.cache(TEMPLATE2); + + return null; + } + }, IllegalArgumentException.class, null); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + ignite.cache(TEMPLATE3); + + return null; + } + }, IllegalArgumentException.class, null); + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java index 71f36cd..b575544 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java @@ -197,6 +197,16 @@ public class IgniteMock implements Ignite { } /** {@inheritDoc} */ + @Override public IgniteCache getOrCreateCache(String cacheName) { + return null; + } + + /** {@inheritDoc} */ + @Override public void addCacheConfiguration(CacheConfiguration cacheCfg) { + // No-op. + } + + /** {@inheritDoc} */ @Override public void destroyCache(String cacheName) { // No-op. } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 00dfc76..18c76b4 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -411,6 +411,8 @@ public class IgniteCacheTestSuite extends TestSuite { suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class); suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class); + suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class); + suite.addTestSuite(IgniteCacheConfigurationTemplateNotFoundTest.class); suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a815911d/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java ---------------------------------------------------------------------- diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java index eebde72..cd4fd73 100644 --- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java +++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java @@ -276,6 +276,20 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea } /** {@inheritDoc} */ + @Override public IgniteCache getOrCreateCache(String cacheName) { + assert g != null; + + return g.getOrCreateCache(cacheName); + } + + /** {@inheritDoc} */ + @Override public void addCacheConfiguration(CacheConfiguration cacheCfg) { + assert g != null; + + g.addCacheConfiguration(cacheCfg); + } + + /** {@inheritDoc} */ @Override public void destroyCache(String cacheName) { assert g != null;