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/08/21 08:40:19 UTC

[10/50] [abbrv] ignite git commit: IGNITE-6067 move initialize() to GridCacheUtils - Fixes #2445.

IGNITE-6067 move initialize() to GridCacheUtils - Fixes #2445.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5578
Commit: 24cfc2aa1a9749c12265427ee701f1e2ca2e4295
Parents: 09d255e
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Tue Aug 15 15:28:05 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Aug 15 15:28:05 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 87 +-----------------
 .../processors/cache/GridCacheUtils.java        | 97 ++++++++++++++++++++
 2 files changed, 99 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/24cfc2aa/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 3406f48..ef27a14 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
@@ -34,9 +34,6 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
-import javax.cache.configuration.Factory;
-import javax.cache.integration.CacheLoader;
-import javax.cache.integration.CacheWriter;
 import javax.management.MBeanServer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -156,12 +153,9 @@ import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_MODE;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
 import static org.apache.ignite.configuration.DeploymentMode.ISOLATED;
 import static org.apache.ignite.configuration.DeploymentMode.PRIVATE;
@@ -244,86 +238,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     private void initialize(CacheConfiguration cfg, CacheObjectContext cacheObjCtx)
         throws IgniteCheckedException {
-        if (cfg.getCacheMode() == null)
-            cfg.setCacheMode(DFLT_CACHE_MODE);
-
-        if (cfg.getNodeFilter() == null)
-            cfg.setNodeFilter(CacheConfiguration.ALL_NODES);
-
-        if (cfg.getAffinity() == null) {
-            if (cfg.getCacheMode() == PARTITIONED) {
-                RendezvousAffinityFunction aff = new RendezvousAffinityFunction();
-
-                cfg.setAffinity(aff);
-            }
-            else if (cfg.getCacheMode() == REPLICATED) {
-                RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 512);
-
-                cfg.setAffinity(aff);
-
-                cfg.setBackups(Integer.MAX_VALUE);
-            }
-            else
-                cfg.setAffinity(new LocalAffinityFunction());
-        }
-        else {
-            if (cfg.getCacheMode() == LOCAL && !(cfg.getAffinity() instanceof LocalAffinityFunction)) {
-                cfg.setAffinity(new LocalAffinityFunction());
-
-                U.warn(log, "AffinityFunction configuration parameter will be ignored for local cache" +
-                    " [cacheName=" + U.maskName(cfg.getName()) + ']');
-            }
-        }
-
-        if (cfg.getCacheMode() == REPLICATED)
-            cfg.setBackups(Integer.MAX_VALUE);
-
-        if (cfg.getQueryParallelism() > 1 && cfg.getCacheMode() != PARTITIONED)
-            throw new IgniteCheckedException("Segmented indices are supported for PARTITIONED mode only.");
-
-        if (cfg.getAffinityMapper() == null)
-            cfg.setAffinityMapper(cacheObjCtx.defaultAffMapper());
+        CU.initializeConfigDefaults(log, cfg, cacheObjCtx);
 
         ctx.igfsHelper().preProcessCacheConfiguration(cfg);
-
-        if (cfg.getRebalanceMode() == null)
-            cfg.setRebalanceMode(ASYNC);
-
-        if (cfg.getAtomicityMode() == null)
-            cfg.setAtomicityMode(CacheConfiguration.DFLT_CACHE_ATOMICITY_MODE);
-
-        if (cfg.getWriteSynchronizationMode() == null)
-            cfg.setWriteSynchronizationMode(PRIMARY_SYNC);
-
-        assert cfg.getWriteSynchronizationMode() != null;
-
-        if (cfg.getCacheStoreFactory() == null) {
-            Factory<CacheLoader> ldrFactory = cfg.getCacheLoaderFactory();
-            Factory<CacheWriter> writerFactory = cfg.isWriteThrough() ? cfg.getCacheWriterFactory() : null;
-
-            if (ldrFactory != null || writerFactory != null)
-                cfg.setCacheStoreFactory(new GridCacheLoaderWriterStoreFactory(ldrFactory, writerFactory));
-        }
-        else {
-            if (cfg.getCacheLoaderFactory() != null)
-                throw new IgniteCheckedException("Cannot set both cache loaded factory and cache store factory " +
-                    "for cache: " + U.maskName(cfg.getName()));
-
-            if (cfg.getCacheWriterFactory() != null)
-                throw new IgniteCheckedException("Cannot set both cache writer factory and cache store factory " +
-                    "for cache: " + U.maskName(cfg.getName()));
-        }
-
-        Collection<QueryEntity> entities = cfg.getQueryEntities();
-
-        if (!F.isEmpty(entities)) {
-            Collection<QueryEntity> normalEntities = new ArrayList<>(entities.size());
-
-            for (QueryEntity entity : entities)
-                normalEntities.add(QueryUtils.normalizeQueryEntity(entity, cfg.isSqlEscapeAll()));
-
-            cfg.clearQueryEntities().setQueryEntities(normalEntities);
-        }
     }
 
     /**
@@ -3961,7 +3878,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    private static class LocalAffinityFunction implements AffinityFunction {
+    static class LocalAffinityFunction implements AffinityFunction {
         /** */
         private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/24cfc2aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index f0b8170..f94cfb5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -36,6 +36,8 @@ import javax.cache.CacheException;
 import javax.cache.configuration.Factory;
 import javax.cache.expiry.Duration;
 import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.integration.CacheLoader;
+import javax.cache.integration.CacheWriter;
 import javax.cache.integration.CacheWriterException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
@@ -46,6 +48,8 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheAtomicUpdateTimeoutException;
 import org.apache.ignite.cache.CachePartialUpdateException;
 import org.apache.ignite.cache.CacheServerNotFoundException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -65,6 +69,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.lang.IgniteInClosureX;
@@ -95,9 +100,13 @@ import org.jsr166.ConcurrentHashMap8;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_MODE;
 import static org.apache.ignite.internal.GridTopic.TOPIC_REPLICATION;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
 
@@ -1614,4 +1623,92 @@ public class GridCacheUtils {
         for (CacheConfiguration ccfg : ccfgs)
             validateCacheName(ccfg.getName());
     }
+
+    /**
+     * @param cfg Initializes cache configuration with proper defaults.
+     * @param cacheObjCtx Cache object context.
+     * @throws IgniteCheckedException If configuration is not valid.
+     */
+    public static void initializeConfigDefaults(IgniteLogger log, CacheConfiguration cfg,
+        CacheObjectContext cacheObjCtx)
+        throws IgniteCheckedException {
+        if (cfg.getCacheMode() == null)
+            cfg.setCacheMode(DFLT_CACHE_MODE);
+
+        if (cfg.getNodeFilter() == null)
+            cfg.setNodeFilter(CacheConfiguration.ALL_NODES);
+
+        if (cfg.getAffinity() == null) {
+            if (cfg.getCacheMode() == PARTITIONED) {
+                RendezvousAffinityFunction aff = new RendezvousAffinityFunction();
+
+                cfg.setAffinity(aff);
+            }
+            else if (cfg.getCacheMode() == REPLICATED) {
+                RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 512);
+
+                cfg.setAffinity(aff);
+
+                cfg.setBackups(Integer.MAX_VALUE);
+            }
+            else
+                cfg.setAffinity(new GridCacheProcessor.LocalAffinityFunction());
+        }
+        else {
+            if (cfg.getCacheMode() == LOCAL && !(cfg.getAffinity() instanceof GridCacheProcessor.LocalAffinityFunction)) {
+                cfg.setAffinity(new GridCacheProcessor.LocalAffinityFunction());
+
+                U.warn(log, "AffinityFunction configuration parameter will be ignored for local cache" +
+                    " [cacheName=" + U.maskName(cfg.getName()) + ']');
+            }
+        }
+
+        if (cfg.getCacheMode() == REPLICATED)
+            cfg.setBackups(Integer.MAX_VALUE);
+
+        if (cfg.getQueryParallelism() > 1 && cfg.getCacheMode() != PARTITIONED)
+            throw new IgniteCheckedException("Segmented indices are supported for PARTITIONED mode only.");
+
+        if (cfg.getAffinityMapper() == null)
+            cfg.setAffinityMapper(cacheObjCtx.defaultAffMapper());
+
+        if (cfg.getRebalanceMode() == null)
+            cfg.setRebalanceMode(ASYNC);
+
+        if (cfg.getAtomicityMode() == null)
+            cfg.setAtomicityMode(CacheConfiguration.DFLT_CACHE_ATOMICITY_MODE);
+
+        if (cfg.getWriteSynchronizationMode() == null)
+            cfg.setWriteSynchronizationMode(PRIMARY_SYNC);
+
+        assert cfg.getWriteSynchronizationMode() != null;
+
+        if (cfg.getCacheStoreFactory() == null) {
+            Factory<CacheLoader> ldrFactory = cfg.getCacheLoaderFactory();
+            Factory<CacheWriter> writerFactory = cfg.isWriteThrough() ? cfg.getCacheWriterFactory() : null;
+
+            if (ldrFactory != null || writerFactory != null)
+                cfg.setCacheStoreFactory(new GridCacheLoaderWriterStoreFactory(ldrFactory, writerFactory));
+        }
+        else {
+            if (cfg.getCacheLoaderFactory() != null)
+                throw new IgniteCheckedException("Cannot set both cache loaded factory and cache store factory " +
+                    "for cache: " + U.maskName(cfg.getName()));
+
+            if (cfg.getCacheWriterFactory() != null)
+                throw new IgniteCheckedException("Cannot set both cache writer factory and cache store factory " +
+                    "for cache: " + U.maskName(cfg.getName()));
+        }
+
+        Collection<QueryEntity> entities = cfg.getQueryEntities();
+
+        if (!F.isEmpty(entities)) {
+            Collection<QueryEntity> normalEntities = new ArrayList<>(entities.size());
+
+            for (QueryEntity entity : entities)
+                normalEntities.add(QueryUtils.normalizeQueryEntity(entity, cfg.isSqlEscapeAll()));
+
+            cfg.clearQueryEntities().setQueryEntities(normalEntities);
+        }
+    }
 }