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 2015/06/16 05:10:55 UTC

[3/4] incubator-ignite git commit: IGNITE-884 - Spring autowiring

IGNITE-884 - Spring autowiring


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

Branch: refs/heads/ignite-884
Commit: 8c0cc4f14294e43ffe1be172f56e5d06fe587c31
Parents: db8eb55
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 15 19:42:18 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 15 19:42:18 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/CacheStartContext.java     | 43 +++-------------
 .../processors/cache/GridCacheContext.java      | 21 ++++++--
 .../GridCacheLoaderWriterStoreFactory.java      | 17 ++++---
 .../processors/cache/GridCacheProcessor.java    | 12 +++--
 .../processors/cache/GridCacheUtils.java        | 53 ++++++++++++++------
 .../store/GridCacheStoreManagerAdapter.java     |  6 ++-
 .../optimized/OptimizedClassDescriptor.java     |  2 +-
 .../loadtests/hashmap/GridCacheTestContext.java |  1 +
 8 files changed, 85 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
index f305a47..3e69d5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 
-import javax.cache.configuration.*;
 import javax.cache.expiry.*;
 import java.util.*;
 
@@ -36,7 +36,7 @@ public class CacheStartContext {
     private ExpiryPolicy expPlc;
 
     /** */
-    private CacheStoreSessionListener[] sesLsnrs;
+    private Collection<CacheStoreSessionListener> sesLsnrs;
 
     /**
      * @param cfg Configuration.
@@ -45,9 +45,9 @@ public class CacheStartContext {
         assert ctx != null;
         assert cfg != null;
 
-        store = create(ctx, cfg.getCacheStoreFactory());
-        expPlc = create(ctx, cfg.getExpiryPolicyFactory());
-        //sesLsnrs = create(ctx, cfg.getCacheStoreSessionListenerFactories());
+        store = CU.create(ctx, cfg.getCacheStoreFactory());
+        expPlc = CU.create(ctx, cfg.getExpiryPolicyFactory());
+        sesLsnrs = CU.create(ctx, cfg.getCacheStoreSessionListenerFactories());
     }
 
     /**
@@ -67,38 +67,7 @@ public class CacheStartContext {
     /**
      * @return Store session listeners.
      */
-    public CacheStoreSessionListener[] storeSessionListeners() {
+    public Collection<CacheStoreSessionListener> storeSessionListeners() {
         return sesLsnrs;
     }
-
-    /**
-     * @param ctx Context.
-     * @param factory Factory.
-     * @return Object.
-     */
-    private <T> T create(GridKernalContext ctx, Factory<T> factory) {
-        T obj = factory != null ? factory.create() : null;
-
-        ctx.resource().autowireSpringBean(obj);
-
-        return obj;
-    }
-
-    /**
-     * @param ctx Context.
-     * @param factories Factories.
-     * @return Objects.
-     */
-    private <T> T[] create(GridKernalContext ctx, Factory<T>[] factories) {
-        Collection<T> col = new ArrayList<>(factories.length);
-
-        for (Factory<T> factory : factories) {
-            T obj = create(ctx, factory);
-
-            if (obj != null)
-                col.add(obj);
-        }
-
-        return (T[])col.toArray();
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 8a4e3b9..f18f62c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -54,12 +55,11 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
-import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.plugin.security.SecurityException;
+import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
-import javax.cache.configuration.*;
 import javax.cache.expiry.*;
 import java.io.*;
 import java.util.*;
@@ -199,6 +199,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Updates allowed flag. */
     private boolean updatesAllowed;
 
+    /** Session listeners. */
+    private Collection<CacheStoreSessionListener> sesLsnrs;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -231,6 +234,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     public GridCacheContext(
         GridKernalContext ctx,
         GridCacheSharedContext sharedCtx,
+        CacheStartContext startCtx,
         CacheConfiguration cacheCfg,
         CacheType cacheType,
         boolean affNode,
@@ -310,13 +314,13 @@ public class GridCacheContext<K, V> implements Externalizable {
 
         plc = cacheType.ioPolicy();
 
-        Factory<ExpiryPolicy> factory = cacheCfg.getExpiryPolicyFactory();
-
-        expiryPlc = factory != null ? factory.create() : null;
+        expiryPlc = startCtx.expiryPolicy();
 
         if (expiryPlc instanceof EternalExpiryPolicy)
             expiryPlc = null;
 
+        sesLsnrs = startCtx.storeSessionListeners();
+
         itHolder = new CacheWeakQueryIteratorsHolder(log);
     }
 
@@ -1029,6 +1033,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Store session listeners.
+     */
+    public Collection<CacheStoreSessionListener> storeSessionListeners() {
+        return sesLsnrs;
+    }
+
+    /**
      * @param p Predicate.
      * @return {@code True} if given predicate is filter for {@code putIfAbsent} operation.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
index 3e27894..82a534c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
@@ -5,9 +5,9 @@
  * 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.
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.configuration.*;
@@ -31,6 +33,9 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
     private static final long serialVersionUID = 0L;
 
     /** */
+    private final GridKernalContext cctx;
+
+    /** */
     private final Factory<CacheLoader<K, V>> ldrFactory;
 
     /** */
@@ -40,8 +45,9 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
      * @param ldrFactory Loader factory.
      * @param writerFactory Writer factory.
      */
-    GridCacheLoaderWriterStoreFactory(@Nullable Factory<CacheLoader<K, V>> ldrFactory,
+    GridCacheLoaderWriterStoreFactory(GridKernalContext cctx, @Nullable Factory<CacheLoader<K, V>> ldrFactory,
         @Nullable Factory<CacheWriter<K, V>> writerFactory) {
+        this.cctx = cctx;
         this.ldrFactory = ldrFactory;
         this.writerFactory = writerFactory;
 
@@ -50,9 +56,8 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
 
     /** {@inheritDoc} */
     @Override public CacheStore<K, V> create() {
-        CacheLoader<K, V> ldr = ldrFactory == null ? null : ldrFactory.create();
-
-        CacheWriter<K, V> writer = writerFactory == null ? null : writerFactory.create();
+        CacheLoader<K, V> ldr = CU.create(cctx, ldrFactory);
+        CacheWriter<K, V> writer = CU.create(cctx, writerFactory);
 
         return new GridCacheLoaderWriterStore<>(ldr, writer);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 b20d32c..f806be7 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
@@ -230,7 +230,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             Factory<CacheWriter> writerFactory = cfg.isWriteThrough() ? cfg.getCacheWriterFactory() : null;
 
             if (ldrFactory != null || writerFactory != null)
-                cfg.setCacheStoreFactory(new GridCacheLoaderWriterStoreFactory(ldrFactory, writerFactory));
+                cfg.setCacheStoreFactory(new GridCacheLoaderWriterStoreFactory(ctx, ldrFactory, writerFactory));
         }
         else {
             if (cfg.getCacheLoaderFactory() != null)
@@ -571,8 +571,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
 
-        sharedCtx = createSharedContext(ctx, CU.startStoreSessionListeners(ctx,
-            ctx.config().getCacheStoreSessionListenerFactories()));
+        Collection<CacheStoreSessionListener> sesLsnrs = CU.create(ctx,
+            ctx.config().getCacheStoreSessionListenerFactories());
+
+        CU.startStoreSessionListeners(ctx, sesLsnrs);
+
+        sharedCtx = createSharedContext(ctx, sesLsnrs);
 
         ctx.performance().add("Disable serializable transactions (set 'txSerializableEnabled' to false)",
             !ctx.config().getTransactionConfiguration().isTxSerializableEnabled());
@@ -1131,6 +1135,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
             ctx,
             sharedCtx,
+            startCtx,
             cfg,
             cacheType,
             ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),
@@ -1261,6 +1266,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheCtx = new GridCacheContext(
                 ctx,
                 sharedCtx,
+                startCtx,
                 cfg,
                 cacheType,
                 ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 3bd2a45..fda05ee 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
@@ -56,7 +56,6 @@ import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheRebalanceMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 import static org.apache.ignite.internal.GridTopic.*;
-import static org.apache.ignite.internal.IgniteNodeAttributes.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 
 /**
@@ -1644,31 +1643,22 @@ public class GridCacheUtils {
      * Creates and starts store session listeners.
      *
      * @param ctx Kernal context.
-     * @param factories Factories.
-     * @return Listeners.
+     * @param sesLsnrs Session listeners.
      * @throws IgniteCheckedException In case of error.
      */
-    public static Collection<CacheStoreSessionListener> startStoreSessionListeners(GridKernalContext ctx,
-        Factory<CacheStoreSessionListener>[] factories) throws IgniteCheckedException {
-        if (factories == null)
-            return null;
-
-        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
-
-        for (Factory<CacheStoreSessionListener> factory : factories) {
-            CacheStoreSessionListener lsnr = factory.create();
+    public static void startStoreSessionListeners(GridKernalContext ctx, Collection<CacheStoreSessionListener> sesLsnrs)
+        throws IgniteCheckedException {
+        if (sesLsnrs == null)
+            return;
 
+        for (CacheStoreSessionListener lsnr : sesLsnrs) {
             if (lsnr != null) {
                 ctx.resource().injectGeneric(lsnr);
 
                 if (lsnr instanceof LifecycleAware)
                     ((LifecycleAware)lsnr).start();
-
-                lsnrs.add(lsnr);
             }
         }
-
-        return lsnrs;
     }
 
     /**
@@ -1690,4 +1680,35 @@ public class GridCacheUtils {
             ctx.resource().cleanupGeneric(lsnr);
         }
     }
+
+    /**
+     * @param ctx Context.
+     * @param factory Factory.
+     * @return Object.
+     */
+    public static <T> T create(GridKernalContext ctx, Factory<T> factory) {
+        T obj = factory != null ? factory.create() : null;
+
+        ctx.resource().autowireSpringBean(obj);
+
+        return obj;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param factories Factories.
+     * @return Objects.
+     */
+    public static <T> Collection<T> create(GridKernalContext ctx, Factory[] factories) {
+        Collection<T> col = new ArrayList<>(factories.length);
+
+        for (Factory<T> factory : factories) {
+            T obj = create(ctx, factory);
+
+            if (obj != null)
+                col.add(obj);
+        }
+
+        return col;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 b4a146a..627aa12 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
@@ -169,9 +169,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 "Persistence store is configured, but both read-through and write-through are disabled.");
         }
 
-        sesLsnrs = CU.startStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
+        sesLsnrs = cctx.storeSessionListeners();
 
-        if (sesLsnrs == null) {
+        if (sesLsnrs != null)
+            CU.startStoreSessionListeners(cctx.kernalContext(), sesLsnrs);
+        else {
             sesLsnrs = cctx.shared().storeSessionListeners();
 
             globalSesLsnrs = true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index 05ef534..b66de27 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -531,7 +531,7 @@ class OptimizedClassDescriptor {
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     void write(OptimizedObjectOutputStream out, Object obj) throws IOException {
-        out.write(type);
+        out.writeByte(type);
 
         switch (type) {
             case BYTE:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 77d3905..efeb8f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -57,6 +57,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheIoManager(),
                 null
             ),
+            new CacheStartContext(ctx, defaultCacheConfiguration()),
             defaultCacheConfiguration(),
             CacheType.USER,
             true,