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 2015/01/13 16:21:51 UTC

[1/7] incubator-ignite git commit: ignite-45: wip

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1 bb32d468d -> 018a9b0cd


ignite-45: wip


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

Branch: refs/heads/ignite-1
Commit: ef624c58cf434469d58efa8b6e06e576e24c4a55
Parents: 180720f
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jan 6 15:45:00 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jan 6 15:45:00 2015 +0300

----------------------------------------------------------------------
 .../services/javax.cache.spi.CachingProvider    |   1 +
 .../org/apache/ignite/IgniteCacheMXBean.java    |  64 ++++
 .../org/apache/ignite/IgniteCacheManager.java   | 311 +++++++++++++++++++
 .../apache/ignite/IgniteCachingProvider.java    | 157 ++++++++++
 .../processors/cache/IgniteCacheProxy.java      |   6 +-
 .../grid/cache/GridCacheConfiguration.java      |  11 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |   2 +-
 .../cache/IgniteCachingProviderSelfTest.java    | 123 ++++++++
 8 files changed, 670 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/META-INF/services/javax.cache.spi.CachingProvider
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/META-INF/services/javax.cache.spi.CachingProvider b/modules/core/src/main/java/META-INF/services/javax.cache.spi.CachingProvider
new file mode 100644
index 0000000..eb232dc
--- /dev/null
+++ b/modules/core/src/main/java/META-INF/services/javax.cache.spi.CachingProvider
@@ -0,0 +1 @@
+org.apache.ignite.IgniteCachingProvider

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
new file mode 100644
index 0000000..0200717
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
@@ -0,0 +1,64 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.management.*;
+
+/**
+ *
+ */
+public class IgniteCacheMXBean implements CacheMXBean {
+    /** */
+    private final Cache<?, ?> cache;
+
+    /**
+     * @param cache Cache.
+     */
+    public IgniteCacheMXBean(Cache<?, ?> cache) {
+        this.cache = cache;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getKeyType() {
+        return cache.getConfiguration(CompleteConfiguration.class).getKeyType().getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getValueType() {
+        return cache.getConfiguration(CompleteConfiguration.class).getValueType().getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isReadThrough() {
+        return cache.getConfiguration(CompleteConfiguration.class).isReadThrough();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWriteThrough() {
+        return cache.getConfiguration(CompleteConfiguration.class).isWriteThrough();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isStoreByValue() {
+        return cache.getConfiguration(CompleteConfiguration.class).isStoreByValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isStatisticsEnabled() {
+        return cache.getConfiguration(CompleteConfiguration.class).isStatisticsEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isManagementEnabled() {
+        return cache.getConfiguration(CompleteConfiguration.class).isManagementEnabled();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
new file mode 100644
index 0000000..7cfeca1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
@@ -0,0 +1,311 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite;
+
+import org.apache.ignite.configuration.*;
+import org.gridgain.grid.cache.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.spi.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class IgniteCacheManager implements CacheManager {
+    /** */
+    private final Map<String, Ignite> igniteMap = new HashMap<>();
+
+    /** */
+    private final URI uri;
+
+    /** */
+    private final CachingProvider cachingProvider;
+
+    /** */
+    private final ClassLoader clsLdr;
+
+    /** */
+    private final AtomicBoolean closed = new AtomicBoolean();
+
+    /**
+     * @param uri Uri.
+     * @param cachingProvider Caching provider.
+     * @param clsLdr Class loader.
+     */
+    public IgniteCacheManager(URI uri, CachingProvider cachingProvider, ClassLoader clsLdr) {
+        this.uri = uri;
+        this.cachingProvider = cachingProvider;
+        this.clsLdr = clsLdr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CachingProvider getCachingProvider() {
+        return cachingProvider;
+    }
+
+    /** {@inheritDoc} */
+    @Override public URI getURI() {
+        return uri;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClassLoader getClassLoader() {
+        return clsLdr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Properties getProperties() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V, C extends Configuration<K, V>> Cache<K, V> createCache(String cacheName, C cacheCfg)
+        throws IllegalArgumentException {
+        ensureNotClosed();
+
+        if (cacheCfg == null)
+            throw new NullPointerException();
+
+        if (!(cacheCfg instanceof CompleteConfiguration))
+            throw new UnsupportedOperationException("Configuration is not supported: " + cacheCfg);
+
+        if (cacheCfg instanceof GridCacheConfiguration) {
+            String cfgCacheName = ((GridCacheConfiguration)cacheCfg).getName();
+
+            if (cfgCacheName != null && !cacheName.equals(cfgCacheName))
+                throw new IllegalArgumentException();
+
+            cacheCfg = (C)new GridCacheConfiguration((GridCacheConfiguration)cacheCfg);
+
+            ((GridCacheConfiguration)cacheCfg).setName(cacheName);
+        }
+
+        Ignite ignite;
+
+        synchronized (igniteMap) {
+            if (igniteMap.containsKey(cacheName))
+                throw new CacheException("Cache already exists [cacheName=" + cacheName + ", manager=" + uri + ']');
+
+            if (uri.equals(cachingProvider.getDefaultURI())) {
+                IgniteConfiguration cfg = new IgniteConfiguration();
+                cfg.setGridName("grid-for-" + cacheName);
+
+                cfg.setCacheConfiguration(new GridCacheConfiguration((CompleteConfiguration)cacheCfg));
+
+                cfg.getCacheConfiguration()[0].setName(cacheName);
+
+                try {
+                    ignite = Ignition.start(cfg);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new CacheException(e);
+                }
+            }
+            else
+                throw new UnsupportedOperationException();
+
+            igniteMap.put(cacheName, ignite);
+        }
+
+        return ignite.jcache(cacheName);
+    }
+
+    /**
+     * @param cacheName Cache name.
+     */
+    private <K, V> IgniteCache<K, V> findCache(String cacheName) {
+        Ignite ignite;
+
+        synchronized (igniteMap) {
+            ignite = igniteMap.get(cacheName);
+        }
+
+        if (ignite == null)
+            return null;
+
+        return ignite.jcache(cacheName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> Cache<K, V> getCache(String cacheName, Class<K> keyType, Class<V> valType) {
+        ensureNotClosed();
+
+        Cache<K, V> cache = findCache(cacheName);
+
+        if (cache != null) {
+            if(!keyType.isAssignableFrom(cache.getConfiguration(Configuration.class).getKeyType()))
+                throw new ClassCastException();
+
+            if(!valType.isAssignableFrom(cache.getConfiguration(Configuration.class).getValueType()))
+                throw new ClassCastException();
+        }
+
+        return cache;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> Cache<K, V> getCache(String cacheName) {
+        ensureNotClosed();
+
+        IgniteCache<K, V> cache = findCache(cacheName);
+
+        if (cache != null) {
+            if(cache.getConfiguration(Configuration.class).getKeyType() != Object.class)
+                throw new IllegalArgumentException();
+
+            if(cache.getConfiguration(Configuration.class).getValueType() != Object.class)
+                throw new IllegalArgumentException();
+        }
+
+        return cache;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterable<String> getCacheNames() {
+        ensureNotClosed();
+
+        Collection<String> res;
+
+        synchronized (igniteMap) {
+            res = new ArrayList<>(igniteMap.keySet());
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void destroyCache(String cacheName) {
+        ensureNotClosed();
+
+        if (cacheName == null)
+            throw new NullPointerException();
+
+        Ignite ignite;
+
+        synchronized (igniteMap) {
+            ignite = igniteMap.remove(cacheName);
+        }
+
+        if (ignite != null) {
+            try {
+                ignite.close();
+            }
+            catch (Exception ignored) {
+
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void enableManagement(String cacheName, boolean enabled) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void enableStatistics(String cacheName, boolean enabled) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     *
+     */
+    private void ensureNotClosed() throws IllegalStateException {
+        if (closed.get())
+            throw new IllegalStateException("Cache manager are closed [uri=" + uri + ", classLoader=" + clsLdr + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        if (closed.compareAndSet(false, true)) {
+            Ignite[] ignites;
+
+            synchronized (igniteMap) {
+                ignites = igniteMap.values().toArray(new Ignite[igniteMap.values().size()]);
+            }
+
+            for (Ignite ignite : ignites) {
+                try {
+                    ignite.close();
+                }
+                catch (Exception ignored) {
+                    // Ignore any exceptions according to javadoc of javax.cache.CacheManager#close()
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClosed() {
+        return closed.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        if(clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+//        if(clazz.isAssignableFrom(ignite.getClass()))
+//            return clazz.cast(ignite);
+
+        throw new IllegalArgumentException();
+    }
+
+//    /**
+//     *
+//     */
+//    private static class Future<T> {
+//        /** */
+//        private volatile T res;
+//
+//        /** */
+//        private volatile Throwable e;
+//
+//        public T get() throws CacheException {
+//            if (res == null && e == null) {
+//                synchronized (this) {
+//                    try {
+//                        while (res == null && e == null)
+//                            wait();
+//                    }
+//                    catch (InterruptedException e) {
+//                        Thread.currentThread().interrupt();
+//
+//                        throw new RuntimeException(e);
+//                    }
+//                }
+//            }
+//
+//            if (res != null)
+//                return res;
+//
+//            assert e != null;
+//
+//            throw new CacheException(e);
+//        }
+//
+//        public synchronized void setException(Throwable e) {
+//            this.e = e;
+//
+//            notifyAll();
+//        }
+//
+//        public synchronized void setCacheManager(T res) {
+//            assert res != null;
+//
+//            this.res = res;
+//
+//            notifyAll();
+//        }
+//    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
new file mode 100644
index 0000000..eab4cf9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
@@ -0,0 +1,157 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite;
+
+import org.gridgain.grid.kernal.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.spi.*;
+import java.net.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteCachingProvider implements CachingProvider {
+    /** */
+    private static final URI DEFAULT_URI;
+
+    static {
+        URI uri = null;
+
+        try {
+            URL dfltCfgURL = U.resolveGridGainUrl(GridGainEx.DFLT_CFG);
+            if (dfltCfgURL != null)
+                uri = dfltCfgURL.toURI();
+        }
+        catch (URISyntaxException ignored) {
+
+        }
+
+        if (uri == null)
+            uri = URI.create("ignite://default");
+
+        DEFAULT_URI = uri;
+    }
+
+    /** */
+    private final Map<ClassLoader, Map<URI, IgniteCacheManager>> cacheManagers = new WeakHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public CacheManager getCacheManager(@Nullable URI uri, ClassLoader clsLdr, Properties props) {
+        if (uri == null)
+            uri = getDefaultURI();
+
+        if (clsLdr == null)
+            clsLdr = getDefaultClassLoader();
+
+        synchronized (cacheManagers) {
+            Map<URI, IgniteCacheManager> uriMap = cacheManagers.get(clsLdr);
+
+            if (uriMap == null) {
+                uriMap = new HashMap<>();
+
+                cacheManagers.put(clsLdr, uriMap);
+            }
+
+            IgniteCacheManager mgr = uriMap.get(uri);
+
+            if (mgr == null) {
+                mgr = new IgniteCacheManager(uri, this, clsLdr);
+
+                uriMap.put(uri, mgr);
+            }
+
+            return mgr;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClassLoader getDefaultClassLoader() {
+        return getClass().getClassLoader();
+    }
+
+    /** {@inheritDoc} */
+    @Override public URI getDefaultURI() {
+        return DEFAULT_URI;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Properties getDefaultProperties() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheManager getCacheManager(URI uri, ClassLoader clsLdr) {
+        return getCacheManager(uri, clsLdr, getDefaultProperties());
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheManager getCacheManager() {
+        return getCacheManager(getDefaultURI(), getDefaultClassLoader());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        Collection<IgniteCacheManager> mgrs = new ArrayList<>();
+
+        synchronized (cacheManagers) {
+            for (Map<URI, IgniteCacheManager> uriMap : cacheManagers.values())
+                mgrs.addAll(uriMap.values());
+
+            cacheManagers.clear();
+        }
+
+        for (IgniteCacheManager mgr : mgrs)
+            mgr.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(ClassLoader clsLdr) {
+        Collection<IgniteCacheManager> mgrs;
+
+        synchronized (cacheManagers) {
+            Map<URI, IgniteCacheManager> uriMap = cacheManagers.remove(clsLdr);
+
+            if (uriMap == null)
+                return;
+
+            mgrs = uriMap.values();
+        }
+
+        for (IgniteCacheManager mgr : mgrs)
+            mgr.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(URI uri, ClassLoader clsLdr) {
+        IgniteCacheManager mgr;
+
+        synchronized (cacheManagers) {
+            Map<URI, IgniteCacheManager> uriMap = cacheManagers.get(clsLdr);
+
+            if (uriMap == null)
+                return;
+
+            mgr = uriMap.remove(uri);
+        }
+
+        if (mgr != null)
+            mgr.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isSupported(OptionalFeature optionalFeature) {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index a985fde..df9bc41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -88,10 +88,12 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
 
     /** {@inheritDoc} */
     @Override public <C extends Configuration<K, V>> C getConfiguration(Class<C> clazz) {
-        if (!clazz.equals(GridCacheConfiguration.class))
+        GridCacheConfiguration cfg = ctx.config();
+
+        if (!clazz.isAssignableFrom(cfg.getClass()))
             throw new IllegalArgumentException();
 
-        return (C)ctx.config();
+        return clazz.cast(cfg);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
index 5a9a675..f92c10a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
@@ -331,9 +331,16 @@ public class GridCacheConfiguration extends MutableConfiguration {
     /**
      * Copy constructor.
      *
-     * @param cc Configuration to copy.
+     * @param cfg Configuration to copy.
      */
-    public GridCacheConfiguration(GridCacheConfiguration cc) {
+    public GridCacheConfiguration(CompleteConfiguration cfg) {
+        super(cfg);
+
+        if (!(cfg instanceof GridCacheConfiguration))
+            return;
+
+        GridCacheConfiguration cc = (GridCacheConfiguration)cfg;
+
         /*
          * NOTE: MAKE SURE TO PRESERVE ALPHABETIC ORDER!
          * ==============================================

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index 555ab55..24b92dd 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -103,7 +103,7 @@ import static org.apache.ignite.plugin.segmentation.GridSegmentationPolicy.*;
  */
 public class GridGainEx {
     /** Default configuration path relative to GridGain home. */
-    private static final String DFLT_CFG = "config/default-config.xml";
+    public static final String DFLT_CFG = "config/default-config.xml";
 
     /** Map of named grids. */
     private static final ConcurrentMap<Object, GridNamedInstance> grids = new ConcurrentHashMap8<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef624c58/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
new file mode 100644
index 0000000..c224ec2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
@@ -0,0 +1,123 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import com.google.common.collect.*;
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.gridgain.grid.cache.*;
+
+import javax.cache.*;
+import javax.cache.spi.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteCachingProviderSelfTest extends IgniteCacheAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheMode cacheMode() {
+        return GridCacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheAtomicityMode atomicityMode() {
+        return GridCacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheDistributionMode distributionMode() {
+        return GridCacheDistributionMode.PARTITIONED_ONLY;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTestGridName(int idx) {
+        assert idx == 0;
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        assert gridName == null;
+
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        GridCacheConfiguration cache1 = cacheConfiguration(null);
+        cache1.setName("cache1");
+
+        GridCacheConfiguration cache2 = cacheConfiguration(null);
+        cache2.setName("cache2");
+
+        cfg.setCacheConfiguration(cacheConfiguration(null), cache1, cache2);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        // No-op. Disabling start of ignite.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    public void testStartIgnite() {
+        CachingProvider cachingProvider = Caching.getCachingProvider();
+
+        assert cachingProvider instanceof IgniteCachingProvider;
+
+        CacheManager cacheMgr = cachingProvider.getCacheManager();
+
+        assertEquals(Collections.<String>emptySet(), Sets.newHashSet(cacheMgr.getCacheNames()));
+
+        Cache<Integer, String> cacheA = cacheMgr.createCache("a", new GridCacheConfiguration());
+
+        cacheA.put(1, "1");
+
+        assertEquals("1", cacheA.get(1));
+
+        cacheMgr.createCache("b", new GridCacheConfiguration());
+
+        assertEquals(Sets.newHashSet("a", "b"), Sets.newHashSet(cacheMgr.getCacheNames()));
+
+        cacheMgr.destroyCache("a");
+        cacheMgr.destroyCache("b");
+
+        assertEquals(Collections.<String>emptySet(), Sets.newHashSet(cacheMgr.getCacheNames()));
+    }
+
+    /**
+     *
+     */
+    public void testCloseManager() throws Exception {
+        startGridsMultiThreaded(1);
+
+        CachingProvider cachingProvider = Caching.getCachingProvider();
+
+        assert cachingProvider instanceof IgniteCachingProvider;
+
+        CacheManager cacheMgr = cachingProvider.getCacheManager();
+
+        cachingProvider.close();
+
+        assertNotSame(cacheMgr, cachingProvider.getCacheManager());
+    }
+}


[2/7] incubator-ignite git commit: ignite-45: wip

Posted by sb...@apache.org.
ignite-45: wip


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

Branch: refs/heads/ignite-1
Commit: b03a4835cd4f6329ecab3e68e0948e579e17c99f
Parents: ef624c5
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Jan 9 12:52:02 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Jan 9 12:52:02 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteCacheManager.java   | 26 +++++++++++++++++---
 .../apache/ignite/IgniteCachingProvider.java    | 20 ++++++++++++++-
 .../processors/cache/IgniteCacheProxy.java      | 20 ++++++++++++---
 3 files changed, 57 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b03a4835/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
index 7cfeca1..a7dab9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
@@ -77,6 +77,9 @@ public class IgniteCacheManager implements CacheManager {
         if (cacheCfg == null)
             throw new NullPointerException();
 
+        if (cacheName == null)
+            throw new NullPointerException();
+
         if (!(cacheCfg instanceof CompleteConfiguration))
             throw new UnsupportedOperationException("Configuration is not supported: " + cacheCfg);
 
@@ -86,7 +89,7 @@ public class IgniteCacheManager implements CacheManager {
             if (cfgCacheName != null && !cacheName.equals(cfgCacheName))
                 throw new IllegalArgumentException();
 
-            cacheCfg = (C)new GridCacheConfiguration((GridCacheConfiguration)cacheCfg);
+            cacheCfg = (C)new GridCacheConfiguration((CompleteConfiguration)cacheCfg);
 
             ((GridCacheConfiguration)cacheCfg).setName(cacheName);
         }
@@ -175,13 +178,22 @@ public class IgniteCacheManager implements CacheManager {
     @Override public Iterable<String> getCacheNames() {
         ensureNotClosed();
 
-        Collection<String> res;
+        String[] resArr;
 
         synchronized (igniteMap) {
-            res = new ArrayList<>(igniteMap.keySet());
+            resArr = igniteMap.keySet().toArray(new String[igniteMap.keySet().size()]);
         }
 
-        return res;
+        return Arrays.asList(resArr);
+    }
+
+    /**
+     * @param ignite Ignite.
+     */
+    public boolean isManagedIgnite(Ignite ignite) {
+        synchronized (igniteMap) {
+            return igniteMap.values().contains(ignite);
+        }
     }
 
     /** {@inheritDoc} */
@@ -209,11 +221,17 @@ public class IgniteCacheManager implements CacheManager {
 
     /** {@inheritDoc} */
     @Override public void enableManagement(String cacheName, boolean enabled) {
+        if (cacheName == null)
+            throw new NullPointerException();
+
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void enableStatistics(String cacheName, boolean enabled) {
+        if (cacheName == null)
+            throw new NullPointerException();
+
         throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b03a4835/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
index eab4cf9..e67434e 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
@@ -66,7 +66,7 @@ public class IgniteCachingProvider implements CachingProvider {
 
             IgniteCacheManager mgr = uriMap.get(uri);
 
-            if (mgr == null) {
+            if (mgr == null || mgr.isClosed()) {
                 mgr = new IgniteCacheManager(uri, this, clsLdr);
 
                 uriMap.put(uri, mgr);
@@ -101,6 +101,24 @@ public class IgniteCachingProvider implements CachingProvider {
         return getCacheManager(getDefaultURI(), getDefaultClassLoader());
     }
 
+    /**
+     * @param cache Cache.
+     */
+    public CacheManager findManager(IgniteCache<?,?> cache) {
+        Ignite ignite = cache.ignite();
+
+        synchronized (cacheManagers) {
+            for (Map<URI, IgniteCacheManager> map : cacheManagers.values()) {
+                for (IgniteCacheManager manager : map.values()) {
+                    if (manager.isManagedIgnite(ignite))
+                        return manager;
+                }
+            }
+        }
+
+        return null;
+    }
+
     /** {@inheritDoc} */
     @Override public void close() {
         Collection<IgniteCacheManager> mgrs = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b03a4835/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index df9bc41..7d0bf2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -428,8 +428,14 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
 
     /** {@inheritDoc} */
     @Override public boolean containsKey(K key) {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+        GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
+
+        try {
+            return delegate.containsKey(key);
+        }
+        finally {
+            gate.leave(prev);
+        }
     }
 
     /** {@inheritDoc} */
@@ -741,8 +747,14 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
 
     /** {@inheritDoc} */
     @Override public CacheManager getCacheManager() {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+        IgniteCachingProvider provider = (IgniteCachingProvider)Caching.getCachingProvider(
+            IgniteCachingProvider.class.getName(),
+            IgniteCachingProvider.class.getClassLoader());
+
+        if (provider == null)
+            return null;
+
+        return provider.findManager(this);
     }
 
     /** {@inheritDoc} */


[3/7] incubator-ignite git commit: ignite-45: wip

Posted by sb...@apache.org.
ignite-45: wip


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

Branch: refs/heads/ignite-1
Commit: 5190b12171bec52727fd40f6c69c8a3c5ac1112f
Parents: b03a483
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Jan 9 17:47:52 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Jan 9 17:47:52 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteCacheManager.java   | 32 ++++++++++++++------
 .../apache/ignite/IgniteCachingProvider.java    |  7 +++--
 .../org/gridgain/grid/cache/GridCacheEntry.java |  3 +-
 .../processors/cache/GridCacheEntryImpl.java    |  8 +++++
 .../cache/GridCacheEvictionEntry.java           |  8 +++++
 .../cache/GridCacheFilterEvaluationEntry.java   |  8 +++++
 .../cache/query/GridCacheQueryManager.java      |  8 +++++
 .../GridCacheContinuousQueryEntry.java          |  8 +++++
 .../cache/eviction/GridCacheMockEntry.java      |  8 +++++
 9 files changed, 78 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
index a7dab9b..2f0073f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
@@ -36,17 +36,22 @@ public class IgniteCacheManager implements CacheManager {
     private final ClassLoader clsLdr;
 
     /** */
+    private final Properties props;
+
+    /** */
     private final AtomicBoolean closed = new AtomicBoolean();
 
     /**
      * @param uri Uri.
      * @param cachingProvider Caching provider.
      * @param clsLdr Class loader.
+     * @param props
      */
-    public IgniteCacheManager(URI uri, CachingProvider cachingProvider, ClassLoader clsLdr) {
+    public IgniteCacheManager(URI uri, CachingProvider cachingProvider, ClassLoader clsLdr, Properties props) {
         this.uri = uri;
         this.cachingProvider = cachingProvider;
         this.clsLdr = clsLdr;
+        this.props = props;
     }
 
     /** {@inheritDoc} */
@@ -66,7 +71,7 @@ public class IgniteCacheManager implements CacheManager {
 
     /** {@inheritDoc} */
     @Override public Properties getProperties() {
-        return null;
+        return props;
     }
 
     /** {@inheritDoc} */
@@ -86,12 +91,15 @@ public class IgniteCacheManager implements CacheManager {
         if (cacheCfg instanceof GridCacheConfiguration) {
             String cfgCacheName = ((GridCacheConfiguration)cacheCfg).getName();
 
-            if (cfgCacheName != null && !cacheName.equals(cfgCacheName))
-                throw new IllegalArgumentException();
-
-            cacheCfg = (C)new GridCacheConfiguration((CompleteConfiguration)cacheCfg);
+            if (cfgCacheName != null) {
+                if (!cacheName.equals(cfgCacheName))
+                    throw new IllegalArgumentException();
+            }
+            else {
+                cacheCfg = (C)new GridCacheConfiguration((CompleteConfiguration)cacheCfg);
 
-            ((GridCacheConfiguration)cacheCfg).setName(cacheName);
+                ((GridCacheConfiguration)cacheCfg).setName(cacheName);
+            }
         }
 
         Ignite ignite;
@@ -176,7 +184,9 @@ public class IgniteCacheManager implements CacheManager {
 
     /** {@inheritDoc} */
     @Override public Iterable<String> getCacheNames() {
-        ensureNotClosed();
+        if (isClosed())
+            return Collections.emptySet(); // javadoc of #getCacheNames() says that IllegalStateException should be
+                                           // thrown but CacheManagerTest.close_cachesEmpty() require empty collection.
 
         String[] resArr;
 
@@ -184,7 +194,7 @@ public class IgniteCacheManager implements CacheManager {
             resArr = igniteMap.keySet().toArray(new String[igniteMap.keySet().size()]);
         }
 
-        return Arrays.asList(resArr);
+        return Collections.unmodifiableCollection(Arrays.asList(resArr));
     }
 
     /**
@@ -221,6 +231,8 @@ public class IgniteCacheManager implements CacheManager {
 
     /** {@inheritDoc} */
     @Override public void enableManagement(String cacheName, boolean enabled) {
+        ensureNotClosed();
+
         if (cacheName == null)
             throw new NullPointerException();
 
@@ -229,6 +241,8 @@ public class IgniteCacheManager implements CacheManager {
 
     /** {@inheritDoc} */
     @Override public void enableStatistics(String cacheName, boolean enabled) {
+        ensureNotClosed();
+
         if (cacheName == null)
             throw new NullPointerException();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
index e67434e..ecc0560 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
@@ -45,6 +45,9 @@ public class IgniteCachingProvider implements CachingProvider {
     }
 
     /** */
+    public static final Properties DFLT_PROPS = new Properties();
+
+    /** */
     private final Map<ClassLoader, Map<URI, IgniteCacheManager>> cacheManagers = new WeakHashMap<>();
 
     /** {@inheritDoc} */
@@ -67,7 +70,7 @@ public class IgniteCachingProvider implements CachingProvider {
             IgniteCacheManager mgr = uriMap.get(uri);
 
             if (mgr == null || mgr.isClosed()) {
-                mgr = new IgniteCacheManager(uri, this, clsLdr);
+                mgr = new IgniteCacheManager(uri, this, clsLdr, props);
 
                 uriMap.put(uri, mgr);
             }
@@ -88,7 +91,7 @@ public class IgniteCachingProvider implements CachingProvider {
 
     /** {@inheritDoc} */
     @Override public Properties getDefaultProperties() {
-        return null;
+        return DFLT_PROPS;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
index d494903..cae7c47 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheEntry.java
@@ -15,6 +15,7 @@ import org.apache.ignite.transactions.*;
 import org.gridgain.grid.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
 import java.util.*;
 import java.util.Map.*;
 
@@ -79,7 +80,7 @@ import java.util.Map.*;
  * @param <K> Key type.
  * @param <V> Value type.
  */
-public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, GridMetadataAware {
+public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, GridMetadataAware, Cache.Entry<K, V> {
     /**
      * Cache projection to which this entry belongs. Note that entry and its
      * parent projections have same flags and filters.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
index c910df2..19ff5b6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
@@ -734,6 +734,14 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        if(clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException();
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return key.hashCode();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
index e81f637..42958eb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
@@ -481,6 +481,14 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        if(clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException();
+    }
+
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return cached.key().hashCode();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
index 9e5644a..4b5cf9d 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
@@ -413,6 +413,14 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        if(clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheFilterEvaluationEntry.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
index 588cac1..ce94ddb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
@@ -2863,6 +2863,14 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         @Override public <V> boolean replaceMeta(String name, V curVal, V newVal) {
             throw new UnsupportedOperationException();
         }
+
+        /** {@inheritDoc} */
+        @Override public <T> T unwrap(Class<T> clazz) {
+            if(clazz.isAssignableFrom(getClass()))
+                return clazz.cast(this);
+
+            throw new IllegalArgumentException();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
index 991573b..bf35bf9 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
@@ -678,6 +678,14 @@ public class GridCacheContinuousQueryEntry<K, V> implements GridCacheEntry<K, V>
     }
 
     /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        if(clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException();
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         boolean b = keyBytes != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5190b121/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheMockEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheMockEntry.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheMockEntry.java
index 6827e66..90e89e6 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheMockEntry.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/eviction/GridCacheMockEntry.java
@@ -343,6 +343,14 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
+    @Override public <T> T unwrap(Class<T> clazz) {
+        if(clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheMockEntry.class, this);
     }


[5/7] incubator-ignite git commit: # IGNITE-45 fix notes found on review.

Posted by sb...@apache.org.
# IGNITE-45 fix notes found on review.


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

Branch: refs/heads/ignite-1
Commit: 7b5d5f44186f2a076ae4130fc1d17cbc4a0ce936
Parents: 5800a75
Author: sevdokimov <se...@gridgain.com>
Authored: Tue Jan 13 18:17:38 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Tue Jan 13 18:17:38 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteCacheMXBean.java    |  22 ++--
 .../org/apache/ignite/IgniteCacheManager.java   |  70 +++--------
 .../apache/ignite/IgniteCachingProvider.java    |  25 ++--
 .../ignite/tck/TCKMBeanServerBuilder.java       | 124 -------------------
 .../grid/kernal/tck/TCKMBeanServerBuilder.java  | 118 ++++++++++++++++++
 5 files changed, 166 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7b5d5f44/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
index 0200717..26cace6 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheMXBean.java
@@ -1,10 +1,18 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+/*
+ * 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;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7b5d5f44/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
index 376c982..fa4530b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
@@ -1,10 +1,18 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+/*
+ * 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;
@@ -361,52 +369,4 @@ public class IgniteCacheManager implements CacheManager {
 
         throw new IllegalArgumentException();
     }
-
-//    /**
-//     *
-//     */
-//    private static class Future<T> {
-//        /** */
-//        private volatile T res;
-//
-//        /** */
-//        private volatile Throwable e;
-//
-//        public T get() throws CacheException {
-//            if (res == null && e == null) {
-//                synchronized (this) {
-//                    try {
-//                        while (res == null && e == null)
-//                            wait();
-//                    }
-//                    catch (InterruptedException e) {
-//                        Thread.currentThread().interrupt();
-//
-//                        throw new RuntimeException(e);
-//                    }
-//                }
-//            }
-//
-//            if (res != null)
-//                return res;
-//
-//            assert e != null;
-//
-//            throw new CacheException(e);
-//        }
-//
-//        public synchronized void setException(Throwable e) {
-//            this.e = e;
-//
-//            notifyAll();
-//        }
-//
-//        public synchronized void setCacheManager(T res) {
-//            assert res != null;
-//
-//            this.res = res;
-//
-//            notifyAll();
-//        }
-//    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7b5d5f44/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
index ecc0560..ea97cb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCachingProvider.java
@@ -1,10 +1,18 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+/*
+ * 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;
@@ -26,6 +34,9 @@ public class IgniteCachingProvider implements CachingProvider {
     /** */
     private static final URI DEFAULT_URI;
 
+    /**
+     *
+     */
     static {
         URI uri = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7b5d5f44/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java b/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java
deleted file mode 100644
index 8123504..0000000
--- a/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.apache.ignite.tck;
-
-import com.sun.jmx.mbeanserver.*;
-
-import javax.management.*;
-
-/**
- *
- */
-public class TCKMBeanServerBuilder extends MBeanServerBuilder {
-    /** {@inheritDoc} */
-    @Override public MBeanServer newMBeanServer(String dfltDomain, MBeanServer outer, MBeanServerDelegate delegate) {
-        MBeanServerDelegate decoratingDelegate = new ServerDelegate(delegate);
-        return JmxMBeanServer.newMBeanServer(dfltDomain, outer,
-            decoratingDelegate, false);
-    }
-
-    /**
-     *
-     */
-    private static class ServerDelegate extends MBeanServerDelegate {
-        /** */
-        private final MBeanServerDelegate delegate;
-
-        /**
-         * Constructor
-         *
-         * @param delegate the provided delegate
-         */
-        ServerDelegate(MBeanServerDelegate delegate) {
-            this.delegate = delegate;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getSpecificationName() {
-            return delegate.getSpecificationName();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public String getSpecificationVersion() {
-            return delegate.getSpecificationVersion();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public String getSpecificationVendor() {
-            return delegate.getSpecificationVendor();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public String getImplementationName() {
-            return delegate.getImplementationName();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public String getImplementationVersion() {
-            return delegate.getImplementationVersion();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public String getImplementationVendor() {
-            return delegate.getImplementationVendor();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public MBeanNotificationInfo[] getNotificationInfo() {
-            return delegate.getNotificationInfo();
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public synchronized void addNotificationListener(NotificationListener listener,
-                                                         NotificationFilter filter,
-                                                         Object handback) throws
-            IllegalArgumentException {
-            delegate.addNotificationListener(listener, filter, handback);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public synchronized void removeNotificationListener(NotificationListener
-                                                                listener,
-                                                            NotificationFilter
-                                                                filter,
-                                                            Object handback) throws
-            ListenerNotFoundException {
-            delegate.removeNotificationListener(listener, filter, handback);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public synchronized void removeNotificationListener(NotificationListener
-                                                                listener) throws
-            ListenerNotFoundException {
-            delegate.removeNotificationListener(listener);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public void sendNotification(Notification notification) {
-            delegate.sendNotification(notification);
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public synchronized String getMBeanServerId() {
-            return System.getProperty("org.jsr107.tck.management.agentId");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7b5d5f44/modules/core/src/main/java/org/gridgain/grid/kernal/tck/TCKMBeanServerBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/tck/TCKMBeanServerBuilder.java b/modules/core/src/main/java/org/gridgain/grid/kernal/tck/TCKMBeanServerBuilder.java
new file mode 100644
index 0000000..fe5e495
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/tck/TCKMBeanServerBuilder.java
@@ -0,0 +1,118 @@
+/*
+ * 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.gridgain.grid.kernal.tck;
+
+import com.sun.jmx.mbeanserver.*;
+
+import javax.management.*;
+
+/**
+ * This class is needed for JCache TCK tests.
+ */
+public class TCKMBeanServerBuilder extends MBeanServerBuilder {
+    /** {@inheritDoc} */
+    @Override public MBeanServer newMBeanServer(String dfltDomain, MBeanServer outer, MBeanServerDelegate delegate) {
+        MBeanServerDelegate decoratingDelegate = new ServerDelegate(delegate);
+        return JmxMBeanServer.newMBeanServer(dfltDomain, outer,
+            decoratingDelegate, false);
+    }
+
+    /**
+     *
+     */
+    private static class ServerDelegate extends MBeanServerDelegate {
+        /** */
+        private final MBeanServerDelegate delegate;
+
+        /**
+         * Constructor
+         *
+         * @param delegate the provided delegate
+         */
+        ServerDelegate(MBeanServerDelegate delegate) {
+            this.delegate = delegate;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getSpecificationName() {
+            return delegate.getSpecificationName();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getSpecificationVersion() {
+            return delegate.getSpecificationVersion();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getSpecificationVendor() {
+            return delegate.getSpecificationVendor();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getImplementationName() {
+            return delegate.getImplementationName();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getImplementationVersion() {
+            return delegate.getImplementationVersion();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getImplementationVendor() {
+            return delegate.getImplementationVendor();
+        }
+
+        /** {@inheritDoc} */
+        @Override public MBeanNotificationInfo[] getNotificationInfo() {
+            return delegate.getNotificationInfo();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized void addNotificationListener(NotificationListener lsnr,
+            NotificationFilter filter,
+            Object handback) throws
+            IllegalArgumentException {
+            delegate.addNotificationListener(lsnr, filter, handback);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized void removeNotificationListener(NotificationListener lsnr,
+            NotificationFilter filter,
+            Object handback) throws
+            ListenerNotFoundException {
+            delegate.removeNotificationListener(lsnr, filter, handback);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized void removeNotificationListener(NotificationListener lsnr)
+            throws ListenerNotFoundException {
+            delegate.removeNotificationListener(lsnr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendNotification(Notification notification) {
+            delegate.sendNotification(notification);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized String getMBeanServerId() {
+            return System.getProperty("org.jsr107.tck.management.agentId");
+        }
+    }
+}


[4/7] incubator-ignite git commit: ignite-45: wip

Posted by sb...@apache.org.
ignite-45: wip


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

Branch: refs/heads/ignite-1
Commit: 5800a75660a534711388367ef9046afb4603d041
Parents: 5190b12
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jan 12 09:35:30 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jan 12 09:35:30 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/IgniteCacheManager.java   | 107 +++++++++++++---
 .../processors/cache/IgniteCacheProxy.java      |   9 +-
 .../ignite/tck/TCKMBeanServerBuilder.java       | 124 +++++++++++++++++++
 3 files changed, 217 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5800a756/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
index 2f0073f..376c982 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCacheManager.java
@@ -10,11 +10,14 @@
 package org.apache.ignite;
 
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
 import org.gridgain.grid.cache.*;
+import org.gridgain.grid.util.typedef.*;
 
 import javax.cache.*;
 import javax.cache.configuration.*;
 import javax.cache.spi.*;
+import javax.management.*;
 import java.net.*;
 import java.util.*;
 import java.util.concurrent.atomic.*;
@@ -24,7 +27,7 @@ import java.util.concurrent.atomic.*;
  */
 public class IgniteCacheManager implements CacheManager {
     /** */
-    private final Map<String, Ignite> igniteMap = new HashMap<>();
+    private final Map<String, IgniteBiTuple<Ignite, IgniteCacheMXBean>> igniteMap = new HashMap<>();
 
     /** */
     private final URI uri;
@@ -45,7 +48,7 @@ public class IgniteCacheManager implements CacheManager {
      * @param uri Uri.
      * @param cachingProvider Caching provider.
      * @param clsLdr Class loader.
-     * @param props
+     * @param props Properties.
      */
     public IgniteCacheManager(URI uri, CachingProvider cachingProvider, ClassLoader clsLdr, Properties props) {
         this.uri = uri;
@@ -102,12 +105,14 @@ public class IgniteCacheManager implements CacheManager {
             }
         }
 
-        Ignite ignite;
+        IgniteCache<K, V> res;
 
         synchronized (igniteMap) {
             if (igniteMap.containsKey(cacheName))
                 throw new CacheException("Cache already exists [cacheName=" + cacheName + ", manager=" + uri + ']');
 
+            Ignite ignite;
+
             if (uri.equals(cachingProvider.getDefaultURI())) {
                 IgniteConfiguration cfg = new IgniteConfiguration();
                 cfg.setGridName("grid-for-" + cacheName);
@@ -126,26 +131,31 @@ public class IgniteCacheManager implements CacheManager {
             else
                 throw new UnsupportedOperationException();
 
-            igniteMap.put(cacheName, ignite);
+            res = ignite.jcache(cacheName);
+
+            igniteMap.put(cacheName, new T2<>(ignite, new IgniteCacheMXBean(res)));
         }
 
-        return ignite.jcache(cacheName);
+        if (((CompleteConfiguration)cacheCfg).isManagementEnabled())
+            enableManagement(cacheName, true);
+
+        return res;
     }
 
     /**
      * @param cacheName Cache name.
      */
     private <K, V> IgniteCache<K, V> findCache(String cacheName) {
-        Ignite ignite;
+        IgniteBiTuple<Ignite, IgniteCacheMXBean> tuple;
 
         synchronized (igniteMap) {
-            ignite = igniteMap.get(cacheName);
+            tuple = igniteMap.get(cacheName);
         }
 
-        if (ignite == null)
+        if (tuple == null)
             return null;
 
-        return ignite.jcache(cacheName);
+        return tuple.get1().jcache(cacheName);
     }
 
     /** {@inheritDoc} */
@@ -202,8 +212,13 @@ public class IgniteCacheManager implements CacheManager {
      */
     public boolean isManagedIgnite(Ignite ignite) {
         synchronized (igniteMap) {
-            return igniteMap.values().contains(ignite);
+            for (IgniteBiTuple<Ignite, IgniteCacheMXBean> tuple : igniteMap.values()) {
+                if (ignite.equals(tuple.get1()))
+                    return true;
+            }
         }
+
+        return false;
     }
 
     /** {@inheritDoc} */
@@ -213,19 +228,48 @@ public class IgniteCacheManager implements CacheManager {
         if (cacheName == null)
             throw new NullPointerException();
 
-        Ignite ignite;
+        IgniteBiTuple<Ignite, IgniteCacheMXBean> tuple;
 
         synchronized (igniteMap) {
-            ignite = igniteMap.remove(cacheName);
+            tuple = igniteMap.remove(cacheName);
         }
 
-        if (ignite != null) {
+        if (tuple != null) {
             try {
-                ignite.close();
+                tuple.get1().close();
             }
             catch (Exception ignored) {
 
             }
+
+            ObjectName objName = getObjectName(cacheName);
+
+            MBeanServer mBeanSrv = tuple.get1().configuration().getMBeanServer();
+
+            for (ObjectName n : mBeanSrv.queryNames(objName, null)) {
+                try {
+                    mBeanSrv.unregisterMBean(n);
+                }
+                catch (Exception ignored) {
+
+                }
+            }
+        }
+    }
+
+    /**
+     * @param cacheName Cache name.
+     */
+    private ObjectName getObjectName(String cacheName) {
+        String mBeanName = "javax.cache:type=CacheConfiguration,CacheManager="
+            + uri.toString().replaceAll(",|:|=|\n", ".")
+            + ",Cache=" + cacheName.replaceAll(",|:|=|\n", ".");
+
+        try {
+            return new ObjectName(mBeanName);
+        }
+        catch (MalformedObjectNameException e) {
+            throw new CacheException("Failed to create MBean name: " + mBeanName, e);
         }
     }
 
@@ -236,7 +280,32 @@ public class IgniteCacheManager implements CacheManager {
         if (cacheName == null)
             throw new NullPointerException();
 
-        throw new UnsupportedOperationException();
+        IgniteBiTuple<Ignite, IgniteCacheMXBean> tuple;
+
+        synchronized (igniteMap) {
+            tuple = igniteMap.get(cacheName);
+        }
+
+        ObjectName objName = getObjectName(cacheName);
+        MBeanServer mBeanSrv = tuple.get1().configuration().getMBeanServer();
+
+        try {
+            if (enabled) {
+                if(mBeanSrv.queryNames(objName, null).isEmpty())
+                    mBeanSrv.registerMBean(tuple.get2(), objName);
+            }
+            else {
+                for (ObjectName n : mBeanSrv.queryNames(objName, null))
+                    mBeanSrv.unregisterMBean(n);
+
+            }
+        }
+        catch (InstanceAlreadyExistsException | InstanceNotFoundException ignored) {
+
+        }
+        catch (MBeanRegistrationException | NotCompliantMBeanException e) {
+            throw new CacheException(e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -260,15 +329,15 @@ public class IgniteCacheManager implements CacheManager {
     /** {@inheritDoc} */
     @Override public void close() {
         if (closed.compareAndSet(false, true)) {
-            Ignite[] ignites;
+            IgniteBiTuple<Ignite, IgniteCacheMXBean>[] ignites;
 
             synchronized (igniteMap) {
-                ignites = igniteMap.values().toArray(new Ignite[igniteMap.values().size()]);
+                ignites = igniteMap.values().toArray(new IgniteBiTuple[igniteMap.values().size()]);
             }
 
-            for (Ignite ignite : ignites) {
+            for (IgniteBiTuple<Ignite, IgniteCacheMXBean> tuple : ignites) {
                 try {
-                    ignite.close();
+                    tuple.get1().close();
                 }
                 catch (Exception ignored) {
                     // Ignore any exceptions according to javadoc of javax.cache.CacheManager#close()

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5800a756/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 7d0bf2d..2f47697 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -747,6 +747,7 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
 
     /** {@inheritDoc} */
     @Override public CacheManager getCacheManager() {
+        // TODO IGNITE-45 (Support start/close/destroy cache correctly)
         IgniteCachingProvider provider = (IgniteCachingProvider)Caching.getCachingProvider(
             IgniteCachingProvider.class.getName(),
             IgniteCachingProvider.class.getClassLoader());
@@ -759,14 +760,14 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
 
     /** {@inheritDoc} */
     @Override public void close() {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+        // TODO IGNITE-45 (Support start/close/destroy cache correctly)
+        getCacheManager().destroyCache(getName());
     }
 
     /** {@inheritDoc} */
     @Override public boolean isClosed() {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+        // TODO IGNITE-45 (Support start/close/destroy cache correctly)
+        return getCacheManager() == null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5800a756/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java b/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java
new file mode 100644
index 0000000..8123504
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/tck/TCKMBeanServerBuilder.java
@@ -0,0 +1,124 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.tck;
+
+import com.sun.jmx.mbeanserver.*;
+
+import javax.management.*;
+
+/**
+ *
+ */
+public class TCKMBeanServerBuilder extends MBeanServerBuilder {
+    /** {@inheritDoc} */
+    @Override public MBeanServer newMBeanServer(String dfltDomain, MBeanServer outer, MBeanServerDelegate delegate) {
+        MBeanServerDelegate decoratingDelegate = new ServerDelegate(delegate);
+        return JmxMBeanServer.newMBeanServer(dfltDomain, outer,
+            decoratingDelegate, false);
+    }
+
+    /**
+     *
+     */
+    private static class ServerDelegate extends MBeanServerDelegate {
+        /** */
+        private final MBeanServerDelegate delegate;
+
+        /**
+         * Constructor
+         *
+         * @param delegate the provided delegate
+         */
+        ServerDelegate(MBeanServerDelegate delegate) {
+            this.delegate = delegate;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getSpecificationName() {
+            return delegate.getSpecificationName();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public String getSpecificationVersion() {
+            return delegate.getSpecificationVersion();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public String getSpecificationVendor() {
+            return delegate.getSpecificationVendor();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public String getImplementationName() {
+            return delegate.getImplementationName();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public String getImplementationVersion() {
+            return delegate.getImplementationVersion();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public String getImplementationVendor() {
+            return delegate.getImplementationVendor();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public MBeanNotificationInfo[] getNotificationInfo() {
+            return delegate.getNotificationInfo();
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public synchronized void addNotificationListener(NotificationListener listener,
+                                                         NotificationFilter filter,
+                                                         Object handback) throws
+            IllegalArgumentException {
+            delegate.addNotificationListener(listener, filter, handback);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public synchronized void removeNotificationListener(NotificationListener
+                                                                listener,
+                                                            NotificationFilter
+                                                                filter,
+                                                            Object handback) throws
+            ListenerNotFoundException {
+            delegate.removeNotificationListener(listener, filter, handback);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public synchronized void removeNotificationListener(NotificationListener
+                                                                listener) throws
+            ListenerNotFoundException {
+            delegate.removeNotificationListener(listener);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public void sendNotification(Notification notification) {
+            delegate.sendNotification(notification);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public synchronized String getMBeanServerId() {
+            return System.getProperty("org.jsr107.tck.management.agentId");
+        }
+    }
+}


[6/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-45' into ignite-1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-45' into ignite-1


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

Branch: refs/heads/ignite-1
Commit: 22796ee9cb21e4fa7cfb3f3886249e2c9e04f699
Parents: bb32d46 7b5d5f4
Author: sevdokimov <se...@gridgain.com>
Authored: Tue Jan 13 18:20:14 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Tue Jan 13 18:20:14 2015 +0300

----------------------------------------------------------------------
 .../services/javax.cache.spi.CachingProvider    |   1 +
 .../org/apache/ignite/IgniteCacheMXBean.java    |  72 ++++
 .../org/apache/ignite/IgniteCacheManager.java   | 372 +++++++++++++++++++
 .../apache/ignite/IgniteCachingProvider.java    | 189 ++++++++++
 .../processors/cache/IgniteCacheProxy.java      |  35 +-
 .../grid/cache/GridCacheConfiguration.java      |  11 +-
 .../org/gridgain/grid/cache/GridCacheEntry.java |   3 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |   2 +-
 .../processors/cache/GridCacheEntryImpl.java    |   8 +
 .../cache/GridCacheEvictionEntry.java           |   8 +
 .../cache/GridCacheFilterEvaluationEntry.java   |   8 +
 .../cache/query/GridCacheQueryManager.java      |   8 +
 .../GridCacheContinuousQueryEntry.java          |   8 +
 .../grid/kernal/tck/TCKMBeanServerBuilder.java  | 118 ++++++
 .../cache/IgniteCachingProviderSelfTest.java    | 123 ++++++
 .../cache/eviction/GridCacheMockEntry.java      |   8 +
 16 files changed, 960 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/22796ee9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------


[7/7] incubator-ignite git commit: # IGNITE-45 fix header.

Posted by sb...@apache.org.
# IGNITE-45 fix header.


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

Branch: refs/heads/ignite-1
Commit: 018a9b0cd154ab5aefc220294b070b3079bb0270
Parents: 22796ee
Author: sevdokimov <se...@gridgain.com>
Authored: Tue Jan 13 18:21:34 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Tue Jan 13 18:21:34 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCachingProviderSelfTest.java    | 22 +++++++++++++-------
 1 file changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/018a9b0c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
index c224ec2..8786c27 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachingProviderSelfTest.java
@@ -1,10 +1,18 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+/*
+ * 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;