You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/09/02 04:00:29 UTC

[01/29] ignite git commit: ignite-1334 Fixed concurrent destroyCache/node stop. Check initFut result in GridDhtPartitionsExchangeFuture.

Repository: ignite
Updated Branches:
  refs/heads/ignite-843 fcf310174 -> 577795018


ignite-1334 Fixed concurrent destroyCache/node stop. Check initFut result in GridDhtPartitionsExchangeFuture.


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

Branch: refs/heads/ignite-843
Commit: afce2699644c8af1e50eb5ef595ed299734c68e5
Parents: 2814d0e
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 1 08:59:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 1 08:59:01 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 27 ++++--
 .../GridDhtPartitionsExchangeFuture.java        | 94 +++++++++++--------
 .../cache/IgniteDynamicCacheAndNodeStop.java    | 95 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  3 +-
 4 files changed, 175 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/afce2699/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 bf203b8..c5f8168 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
@@ -156,6 +156,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** */
     private final Map<String, GridCacheAdapter<?, ?>> caches;
 
+    /** Caches stopped from onKernalStop callback. */
+    private final Map<String, GridCacheAdapter> stoppedCaches = new ConcurrentHashMap<>();
+
     /** Map of proxies. */
     private final Map<String, IgniteCacheProxy<?, ?>> jCacheProxies;
 
@@ -893,14 +896,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = stoppedCaches.remove(maskNull(cacheName));
 
             if (cache != null)
                 stopCache(cache, cancel);
         }
 
-        for (GridCacheAdapter<?, ?> cache : caches.values())
-            stopCache(cache, cancel);
+        for (GridCacheAdapter<?, ?> cache : stoppedCaches.values()) {
+            if (cache == stoppedCaches.remove(maskNull(cache.name())))
+                stopCache(cache, cancel);
+        }
 
         List<? extends GridCacheSharedManager<?, ?>> mgrs = sharedCtx.managers();
 
@@ -932,15 +937,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheStartedLatch.countDown();
 
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.get(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+
+            if (cache != null) {
+                stoppedCaches.put(maskNull(cacheName), cache);
 
-            if (cache != null)
                 onKernalStop(cache, cancel);
+            }
         }
 
         for (Map.Entry<String, GridCacheAdapter<?, ?>> entry : caches.entrySet()) {
-            if (!stopSeq.contains(entry.getKey()))
+            GridCacheAdapter<?, ?> cache = entry.getValue();
+
+            if (cache == caches.remove(entry.getKey())) {
+                stoppedCaches.put(entry.getKey(), cache);
+
                 onKernalStop(entry.getValue(), cancel);
+            }
         }
 
         List<? extends GridCacheSharedManager<?, ?>> sharedMgrs = sharedCtx.managers();
@@ -3457,4 +3470,4 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             // No-op.
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/afce2699/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 414a152..865bbdc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1146,52 +1146,54 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         }
         else {
             initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-                @Override public void apply(IgniteInternalFuture<Boolean> t) {
+                @Override public void apply(IgniteInternalFuture<Boolean> f) {
                     try {
-                        if (!t.get()) // Just to check if there was an error.
+                        if (!f.get())
                             return;
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize exchange future: " + this, e);
 
-                        ClusterNode loc = cctx.localNode();
+                        return;
+                    }
 
-                        singleMsgs.put(nodeId, msg);
+                    ClusterNode loc = cctx.localNode();
 
-                        boolean match = true;
+                    singleMsgs.put(nodeId, msg);
 
-                        // Check if oldest node has changed.
-                        if (!oldestNode.get().equals(loc)) {
-                            match = false;
+                    boolean match = true;
 
-                            synchronized (mux) {
-                                // Double check.
-                                if (oldestNode.get().equals(loc))
-                                    match = true;
-                            }
+                    // Check if oldest node has changed.
+                    if (!oldestNode.get().equals(loc)) {
+                        match = false;
+
+                        synchronized (mux) {
+                            // Double check.
+                            if (oldestNode.get().equals(loc))
+                                match = true;
                         }
+                    }
 
-                        if (match) {
-                            boolean allReceived;
+                    if (match) {
+                        boolean allReceived;
 
-                            synchronized (rcvdIds) {
-                                if (rcvdIds.add(nodeId))
-                                    updatePartitionSingleMap(msg);
+                        synchronized (rcvdIds) {
+                            if (rcvdIds.add(nodeId))
+                                updatePartitionSingleMap(msg);
 
-                                allReceived = allReceived();
-                            }
+                            allReceived = allReceived();
+                        }
 
-                            // If got all replies, and initialization finished, and reply has not been sent yet.
-                            if (allReceived && ready.get() && replied.compareAndSet(false, true)) {
-                                spreadPartitions();
+                        // If got all replies, and initialization finished, and reply has not been sent yet.
+                        if (allReceived && ready.get() && replied.compareAndSet(false, true)) {
+                            spreadPartitions();
 
-                                onDone(exchId.topologyVersion());
-                            }
-                            else if (log.isDebugEnabled())
-                                log.debug("Exchange future full map is not sent [allReceived=" + allReceived() +
-                                    ", ready=" + ready + ", replied=" + replied.get() + ", init=" + init.get() +
-                                    ", fut=" + this + ']');
+                            onDone(exchId.topologyVersion());
                         }
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to initialize exchange future: " + this, e);
+                        else if (log.isDebugEnabled())
+                            log.debug("Exchange future full map is not sent [allReceived=" + allReceived() +
+                                ", ready=" + ready + ", replied=" + replied.get() + ", init=" + init.get() +
+                                ", fut=" + this + ']');
                     }
                 }
             });
@@ -1254,7 +1256,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         assert exchId.topologyVersion().equals(msg.topologyVersion());
 
         initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-            @Override public void apply(IgniteInternalFuture<Boolean> t) {
+            @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                try {
+                    if (!f.get())
+                        return;
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to initialize exchange future: " + this, e);
+
+                    return;
+                }
+
                 ClusterNode curOldest = oldestNode.get();
 
                 if (!nodeId.equals(curOldest.id())) {
@@ -1343,8 +1355,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         try {
             // Wait for initialization part of this future to complete.
-            initFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> f) {
+            initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                    try {
+                        if (!f.get())
+                            return;
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to initialize exchange future: " + this, e);
+
+                        return;
+                    }
+
                     if (isDone())
                         return;
 
@@ -1571,4 +1593,4 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             "remaining", remaining(),
             "super", super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/afce2699/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
new file mode 100644
index 0000000..a389e1f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheAndNodeStop.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheAndNodeStop extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheAndNodeStop() throws Exception {
+        final Ignite ignite = startGrid(0);
+
+        for (int i = 0; i < 3; i++) {
+            log.info("Iteration: " + i);
+
+            startGrid(1);
+
+            final  CacheConfiguration ccfg = new CacheConfiguration();
+
+            ignite.createCache(ccfg);
+
+            final CyclicBarrier barrier = new CyclicBarrier(2);
+
+            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    barrier.await();
+
+                    ignite.destroyCache(null);
+
+                    return null;
+                }
+            });
+
+            IgniteInternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    barrier.await();
+
+                    stopGrid(1);
+
+                    return null;
+                }
+            });
+
+            fut1.get();
+            fut2.get();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/afce2699/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index e903115..9b9bbba 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -238,7 +238,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
         suite.addTest(new TestSuite(CrossCacheTxRandomOperationsTest.class));
+        suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
 
         return suite;
     }
-}
\ No newline at end of file
+}


[08/29] ignite git commit: IGNITE-1337: Moved "common" CPP project to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/src/java.cpp b/modules/platform/src/main/cpp/common/src/java.cpp
new file mode 100644
index 0000000..0c6524b
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/src/java.cpp
@@ -0,0 +1,2204 @@
+/*
+ * 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.
+ */
+
+#include <cstring>
+#include <string>
+#include <exception>
+
+#include "ignite/common/concurrent.h"
+#include "ignite/common/java.h"
+
+#define IGNITE_SAFE_PROC_NO_ARG(jniEnv, envPtr, type, field) { \
+    JniHandlers* hnds = reinterpret_cast<JniHandlers*>(envPtr); \
+    type hnd = hnds->field; \
+    if (hnd) \
+        hnd(hnds->target); \
+    else \
+        ThrowOnMissingHandler(jniEnv); \
+}
+
+#define IGNITE_SAFE_PROC(jniEnv, envPtr, type, field, ...) { \
+    JniHandlers* hnds = reinterpret_cast<JniHandlers*>(envPtr); \
+    type hnd = hnds->field; \
+    if (hnd) \
+        hnd(hnds->target, __VA_ARGS__); \
+    else \
+        ThrowOnMissingHandler(jniEnv); \
+}
+
+#define IGNITE_SAFE_FUNC(jniEnv, envPtr, type, field, ...) { \
+    JniHandlers* hnds = reinterpret_cast<JniHandlers*>(envPtr); \
+    type hnd = hnds->field; \
+    if (hnd) \
+        return hnd(hnds->target, __VA_ARGS__); \
+    else \
+    { \
+        ThrowOnMissingHandler(jniEnv); \
+        return 0; \
+    }\
+}
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace java
+        {
+            namespace gcc = ignite::common::concurrent;
+
+            /* --- Startup exception. --- */
+            class JvmException : public std::exception {
+                // No-op.
+            };
+
+            /* --- JNI method definitions. --- */
+            struct JniMethod {
+                char* name;
+                char* sign;
+                bool isStatic;
+
+                JniMethod(const char* name, const char* sign, bool isStatic) {
+                    this->name = const_cast<char*>(name);
+                    this->sign = const_cast<char*>(sign);
+                    this->isStatic = isStatic;
+                }
+            };
+
+            /*
+             * Heloper function to copy characters.
+             *
+             * @param src Source.
+             * @return Result.
+             */
+            char* CopyChars(const char* src)
+            {
+                if (src)
+                {
+                    size_t len = strlen(src);
+                    char* dest = new char[len + 1];
+                    strcpy(dest, src);
+                    *(dest + len) = 0;
+                    return dest;
+                }
+                else
+                    return NULL;
+            }
+
+            JniErrorInfo::JniErrorInfo() : code(IGNITE_JNI_ERR_SUCCESS), errCls(NULL), errMsg(NULL)
+            {
+                // No-op.
+            }
+
+            JniErrorInfo::JniErrorInfo(int code, const char* errCls, const char* errMsg) : code(code)
+            {
+                this->errCls = CopyChars(errCls);
+                this->errMsg = CopyChars(errMsg);
+            }
+
+            JniErrorInfo::JniErrorInfo(const JniErrorInfo& other) : code(other.code)
+            {
+                this->errCls = CopyChars(other.errCls);
+                this->errMsg = CopyChars(other.errMsg);
+            }
+
+            JniErrorInfo& JniErrorInfo::operator=(const JniErrorInfo& other)
+            {
+                if (this != &other)
+                {
+                    // 1. Create new instance, exception could occur at this point.
+                    JniErrorInfo tmp(other);
+
+                    // 2. Swap with temp.
+                    int code0 = code;
+                    char* errCls0 = errCls;
+                    char* errMsg0 = errMsg;
+
+                    code = tmp.code;
+                    errCls = tmp.errCls;
+                    errMsg = tmp.errMsg;
+
+                    tmp.code = code0;
+                    tmp.errCls = errCls0;
+                    tmp.errMsg = errMsg0;
+                }
+
+                return *this;
+            }
+
+            JniErrorInfo::~JniErrorInfo()
+            {
+                if (errCls)
+                    delete[] errCls;
+
+                if (errMsg)
+                    delete[] errMsg;
+            }
+
+            const char* C_THROWABLE = "java/lang/Throwable";
+            JniMethod M_THROWABLE_GET_MESSAGE = JniMethod("getMessage", "()Ljava/lang/String;", false);
+            JniMethod M_THROWABLE_PRINT_STACK_TRACE = JniMethod("printStackTrace", "()V", false);
+
+            const char* C_CLASS = "java/lang/Class";
+            JniMethod M_CLASS_GET_NAME = JniMethod("getName", "()Ljava/lang/String;", false);
+
+            const char* C_IGNITE_EXCEPTION = "org/apache/ignite/IgniteException";
+
+            const char* C_PLATFORM_NO_CALLBACK_EXCEPTION = "org/apache/ignite/internal/processors/platform/PlatformNoCallbackException";
+
+            const char* C_PLATFORM_PROCESSOR = "org/apache/ignite/internal/processors/platform/PlatformProcessor";
+            JniMethod M_PLATFORM_PROCESSOR_RELEASE_START = JniMethod("releaseStart", "()V", false);
+            JniMethod M_PLATFORM_PROCESSOR_PROJECTION = JniMethod("projection", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CACHE = JniMethod("cache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE = JniMethod("createCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE = JniMethod("getOrCreateCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_AFFINITY = JniMethod("affinity", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_DATA_STREAMER = JniMethod("dataStreamer", "(Ljava/lang/String;Z)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_TRANSACTIONS = JniMethod("transactions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_COMPUTE = JniMethod("compute", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_MESSAGE = JniMethod("message", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_EVENTS = JniMethod("events", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            
+            const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
+            JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false);
+            JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT = JniMethod("inStreamOutObject", "(IJ)Ljava/lang/Object;", false);
+            JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM = JniMethod("inStreamOutStream", "(IJJ)V", false);
+            JniMethod M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM = JniMethod("inObjectStreamOutStream", "(ILjava/lang/Object;JJ)V", false);
+            JniMethod M_PLATFORM_TARGET_OUT_LONG = JniMethod("outLong", "(I)J", false);
+            JniMethod M_PLATFORM_TARGET_OUT_STREAM = JniMethod("outStream", "(IJ)V", false);
+            JniMethod M_PLATFORM_TARGET_OUT_OBJECT = JniMethod("outObject", "(I)Ljava/lang/Object;", false);
+            JniMethod M_PLATFORM_TARGET_LISTEN_FUTURE = JniMethod("listenFuture", "(JI)V", false);
+            JniMethod M_PLATFORM_TARGET_LISTEN_FOR_OPERATION = JniMethod("listenFutureForOperation", "(JII)V", false);
+
+            const char* C_PLATFORM_CLUSTER_GRP = "org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup";
+            JniMethod M_PLATFORM_CLUSTER_GRP_FOR_OTHERS = JniMethod("forOthers", "(Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;)Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false);
+            JniMethod M_PLATFORM_CLUSTER_GRP_FOR_REMOTES = JniMethod("forRemotes", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false);
+            JniMethod M_PLATFORM_CLUSTER_GRP_FOR_DAEMONS = JniMethod("forDaemons", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false);
+            JniMethod M_PLATFORM_CLUSTER_GRP_FOR_RANDOM = JniMethod("forRandom", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false);
+            JniMethod M_PLATFORM_CLUSTER_GRP_FOR_OLDEST = JniMethod("forOldest", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false);
+            JniMethod M_PLATFORM_CLUSTER_GRP_FOR_YOUNGEST = JniMethod("forYoungest", "()Lorg/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup;", false);
+            JniMethod M_PLATFORM_CLUSTER_GRP_RESET_METRICS = JniMethod("resetMetrics", "()V", false);
+            
+            const char* C_PLATFORM_MESSAGING = "org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging";
+            JniMethod M_PLATFORM_MESSAGING_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/messaging/PlatformMessaging;", false);
+
+            const char* C_PLATFORM_COMPUTE = "org/apache/ignite/internal/processors/platform/compute/PlatformCompute";
+            JniMethod M_PLATFORM_COMPUTE_WITH_NO_FAILOVER = JniMethod("withNoFailover", "()V", false);
+            JniMethod M_PLATFORM_COMPUTE_WITH_TIMEOUT = JniMethod("withTimeout", "(J)V", false);
+            JniMethod M_PLATFORM_COMPUTE_EXECUTE_NATIVE = JniMethod("executeNative", "(JJ)V", false);
+
+            const char* C_PLATFORM_CACHE = "org/apache/ignite/internal/processors/platform/cache/PlatformCache";
+            JniMethod M_PLATFORM_CACHE_WITH_SKIP_STORE = JniMethod("withSkipStore", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false);
+            JniMethod M_PLATFORM_CACHE_WITH_NO_RETRIES = JniMethod("withNoRetries", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false);
+            JniMethod M_PLATFORM_CACHE_WITH_EXPIRY_PLC = JniMethod("withExpiryPolicy", "(JJJ)Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false);
+            JniMethod M_PLATFORM_CACHE_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false);
+            JniMethod M_PLATFORM_CACHE_WITH_KEEP_PORTABLE = JniMethod("withKeepPortable", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCache;", false);
+            JniMethod M_PLATFORM_CACHE_CLEAR = JniMethod("clear", "()V", false);
+            JniMethod M_PLATFORM_CACHE_REMOVE_ALL = JniMethod("removeAll", "()V", false);
+            JniMethod M_PLATFORM_CACHE_ITERATOR = JniMethod("iterator", "()Lorg/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator;", false);
+            JniMethod M_PLATFORM_CACHE_LOCAL_ITERATOR = JniMethod("localIterator", "(I)Lorg/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator;", false);
+            JniMethod M_PLATFORM_CACHE_ENTER_LOCK = JniMethod("enterLock", "(J)V", false);
+            JniMethod M_PLATFORM_CACHE_EXIT_LOCK = JniMethod("exitLock", "(J)V", false);
+            JniMethod M_PLATFORM_CACHE_TRY_ENTER_LOCK = JniMethod("tryEnterLock", "(JJ)Z", false);
+            JniMethod M_PLATFORM_CACHE_CLOSE_LOCK = JniMethod("closeLock", "(J)V", false);
+            JniMethod M_PLATFORM_CACHE_REBALANCE = JniMethod("rebalance", "(J)V", false);
+            JniMethod M_PLATFORM_CACHE_SIZE = JniMethod("size", "(IZ)I", false);
+
+            const char* C_PLATFORM_AFFINITY = "org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity";
+            JniMethod C_PLATFORM_AFFINITY_PARTITIONS = JniMethod("partitions", "()I", false);
+
+            const char* C_PLATFORM_DATA_STREAMER = "org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer";
+            JniMethod M_PLATFORM_DATA_STREAMER_LISTEN_TOPOLOGY = JniMethod("listenTopology", "(J)V", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_GET_ALLOW_OVERWRITE = JniMethod("allowOverwrite", "()Z", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_SET_ALLOW_OVERWRITE = JniMethod("allowOverwrite", "(Z)V", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_GET_SKIP_STORE = JniMethod("skipStore", "()Z", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_SET_SKIP_STORE = JniMethod("skipStore", "(Z)V", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_GET_PER_NODE_BUFFER_SIZE = JniMethod("perNodeBufferSize", "()I", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_SET_PER_NODE_BUFFER_SIZE = JniMethod("perNodeBufferSize", "(I)V", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_GET_PER_NODE_PARALLEL_OPS = JniMethod("perNodeParallelOperations", "()I", false);
+            JniMethod M_PLATFORM_DATA_STREAMER_SET_PER_NODE_PARALLEL_OPS = JniMethod("perNodeParallelOperations", "(I)V", false);
+
+            const char* C_PLATFORM_TRANSACTIONS = "org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions";
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_START = JniMethod("txStart", "(IIJI)J", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_COMMIT = JniMethod("txCommit", "(J)I", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_ROLLBACK = JniMethod("txRollback", "(J)I", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_COMMIT_ASYNC = JniMethod("txCommitAsync", "(JJ)V", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_ROLLBACK_ASYNC = JniMethod("txRollbackAsync", "(JJ)V", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_STATE = JniMethod("txState", "(J)I", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_SET_ROLLBACK_ONLY = JniMethod("txSetRollbackOnly", "(J)Z", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_TX_CLOSE = JniMethod("txClose", "(J)I", false);
+            JniMethod M_PLATFORM_TRANSACTIONS_RESET_METRICS = JniMethod("resetMetrics", "()V", false);
+
+            const char* C_PLATFORM_CACHE_STORE_CALLBACK = "org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback";
+            JniMethod M_PLATFORM_CACHE_STORE_CALLBACK_INVOKE = JniMethod("invoke", "(J)V", false);
+
+            const char* C_PLATFORM_CALLBACK_UTILS = "org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils";
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_CREATE = JniMethod("cacheStoreCreate", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_INVOKE = JniMethod("cacheStoreInvoke", "(JJJLjava/lang/Object;)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_DESTROY = JniMethod("cacheStoreDestroy", "(JJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_SESSION_CREATE = JniMethod("cacheStoreSessionCreate", "(JJ)J", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_ENTRY_FILTER_CREATE = JniMethod("cacheEntryFilterCreate", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_ENTRY_FILTER_APPLY = JniMethod("cacheEntryFilterApply", "(JJJ)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_ENTRY_FILTER_DESTROY = JniMethod("cacheEntryFilterDestroy", "(JJ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CACHE_INVOKE = JniMethod("cacheInvoke", "(JJJ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_MAP = JniMethod("computeTaskMap", "(JJJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_JOB_RESULT = JniMethod("computeTaskJobResult", "(JJJJ)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_REDUCE = JniMethod("computeTaskReduce", "(JJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_COMPLETE = JniMethod("computeTaskComplete", "(JJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_SERIALIZE = JniMethod("computeJobSerialize", "(JJJ)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_CREATE = JniMethod("computeJobCreate", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_EXECUTE = JniMethod("computeJobExecute", "(JJIJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_DESTROY = JniMethod("computeJobDestroy", "(JJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_CANCEL = JniMethod("computeJobCancel", "(JJ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_LSNR_APPLY = JniMethod("continuousQueryListenerApply", "(JJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_CREATE = JniMethod("continuousQueryFilterCreate", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_EVAL = JniMethod("continuousQueryFilterApply", "(JJJ)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_RELEASE = JniMethod("continuousQueryFilterRelease", "(JJ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_TOPOLOGY_UPDATE = JniMethod("dataStreamerTopologyUpdate", "(JJJI)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_STREAM_RECEIVER_INVOKE = JniMethod("dataStreamerStreamReceiverInvoke", "(JJLjava/lang/Object;JZ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_BYTE_RES = JniMethod("futureByteResult", "(JJI)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_BOOL_RES = JniMethod("futureBoolResult", "(JJI)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_SHORT_RES = JniMethod("futureShortResult", "(JJI)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_CHAR_RES = JniMethod("futureCharResult", "(JJI)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_INT_RES = JniMethod("futureIntResult", "(JJI)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_FLOAT_RES = JniMethod("futureFloatResult", "(JJF)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_LONG_RES = JniMethod("futureLongResult", "(JJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_DOUBLE_RES = JniMethod("futureDoubleResult", "(JJD)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_OBJ_RES = JniMethod("futureObjectResult", "(JJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_NULL_RES = JniMethod("futureNullResult", "(JJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_ERR = JniMethod("futureError", "(JJJ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_LIFECYCLE_EVENT = JniMethod("lifecycleEvent", "(JJI)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_MESSAGING_FILTER_CREATE = JniMethod("messagingFilterCreate", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_MESSAGING_FILTER_APPLY = JniMethod("messagingFilterApply", "(JJJ)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_MESSAGING_FILTER_DESTROY = JniMethod("messagingFilterDestroy", "(JJ)V", true);
+            
+            JniMethod M_PLATFORM_CALLBACK_UTILS_EVENT_FILTER_CREATE = JniMethod("eventFilterCreate", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_EVENT_FILTER_APPLY = JniMethod("eventFilterApply", "(JJJ)I", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_EVENT_FILTER_DESTROY = JniMethod("eventFilterDestroy", "(JJ)V", true);
+            
+            JniMethod M_PLATFORM_CALLBACK_UTILS_SERVICE_INIT = JniMethod("serviceInit", "(JJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_SERVICE_EXECUTE = JniMethod("serviceExecute", "(JJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_SERVICE_CANCEL = JniMethod("serviceCancel", "(JJJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_SERVICE_INVOKE_METHOD = JniMethod("serviceInvokeMethod", "(JJJJ)V", true);
+			
+            JniMethod M_PLATFORM_CALLBACK_UTILS_CLUSTER_NODE_FILTER_APPLY = JniMethod("clusterNodeFilterApply", "(JJ)I", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_NODE_INFO = JniMethod("nodeInfo", "(JJ)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_MEMORY_REALLOCATE = JniMethod("memoryReallocate", "(JJI)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_ON_START = JniMethod("onStart", "(JJ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_ON_STOP = JniMethod("onStop", "(J)V", true);
+
+            JniMethod M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_OUT_LONG = JniMethod("extensionCallbackInLongOutLong", "(JIJ)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_LONG_OUT_LONG = JniMethod("extensionCallbackInLongLongOutLong", "(JIJJ)J", true);
+            
+            const char* C_PLATFORM_UTILS = "org/apache/ignite/internal/processors/platform/utils/PlatformUtils";
+            JniMethod M_PLATFORM_UTILS_REALLOC = JniMethod("reallocate", "(JI)V", true);
+            JniMethod M_PLATFORM_UTILS_ERR_DATA = JniMethod("errorData", "(Ljava/lang/Throwable;)[B", true);
+
+            const char* C_PLATFORM_IGNITION = "org/apache/ignite/internal/processors/platform/PlatformIgnition";
+            JniMethod M_PLATFORM_IGNITION_START = JniMethod("start", "(Ljava/lang/String;Ljava/lang/String;IJJ)Lorg/apache/ignite/internal/processors/platform/PlatformProcessor;", true);
+            JniMethod M_PLATFORM_IGNITION_INSTANCE = JniMethod("instance", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformProcessor;", true);
+            JniMethod M_PLATFORM_IGNITION_ENVIRONMENT_POINTER = JniMethod("environmentPointer", "(Ljava/lang/String;)J", true);
+            JniMethod M_PLATFORM_IGNITION_STOP = JniMethod("stop", "(Ljava/lang/String;Z)Z", true);
+            JniMethod M_PLATFORM_IGNITION_STOP_ALL = JniMethod("stopAll", "(Z)V", true);
+            
+            const char* C_PLATFORM_ABSTRACT_QRY_CURSOR = "org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor";
+            JniMethod M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER = JniMethod("iterator", "()V", false);
+            JniMethod M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER_HAS_NEXT = JniMethod("iteratorHasNext", "()Z", false);
+            JniMethod M_PLATFORM_ABSTRACT_QRY_CURSOR_CLOSE = JniMethod("close", "()V", false);
+
+            const char* C_PLATFORM_CONT_QRY = "org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery";
+            JniMethod M_PLATFORM_CONT_QRY_CLOSE = JniMethod("close", "()V", false);
+            JniMethod M_PLATFORM_CONT_QRY_GET_INITIAL_QUERY_CURSOR = JniMethod("getInitialQueryCursor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+
+            const char* C_PLATFORM_EVENTS = "org/apache/ignite/internal/processors/platform/events/PlatformEvents";
+            JniMethod M_PLATFORM_EVENTS_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/events/PlatformEvents;", false);
+            JniMethod M_PLATFORM_EVENTS_STOP_LOCAL_LISTEN = JniMethod("stopLocalListen", "(J)Z", false);
+            JniMethod M_PLATFORM_EVENTS_LOCAL_LISTEN = JniMethod("localListen", "(JI)V", false);
+            JniMethod M_PLATFORM_EVENTS_IS_ENABLED = JniMethod("isEnabled", "(I)Z", false);
+            
+            const char* C_PLATFORM_SERVICES = "org/apache/ignite/internal/processors/platform/services/PlatformServices";
+			JniMethod M_PLATFORM_SERVICES_WITH_ASYNC = JniMethod("withAsync", "()Lorg/apache/ignite/internal/processors/platform/services/PlatformServices;", false);
+			JniMethod M_PLATFORM_SERVICES_WITH_SERVER_KEEP_PORTABLE = JniMethod("withServerKeepPortable", "()Lorg/apache/ignite/internal/processors/platform/services/PlatformServices;", false);
+			JniMethod M_PLATFORM_SERVICES_CANCEL = JniMethod("cancel", "(Ljava/lang/String;)V", false);
+			JniMethod M_PLATFORM_SERVICES_CANCEL_ALL = JniMethod("cancelAll", "()V", false);
+			JniMethod M_PLATFORM_SERVICES_SERVICE_PROXY = JniMethod("dotNetServiceProxy", "(Ljava/lang/String;Z)Ljava/lang/Object;", false);
+
+            /* STATIC STATE. */
+            gcc::CriticalSection JVM_LOCK;
+            JniJvm JVM;
+            bool PRINT_EXCEPTION = false;
+
+            /* HELPER METHODS. */
+
+            /*
+             * Throw exception to Java in case of missing callback pointer. It means that callback is not implemented in
+             * native platform and Java -> platform operation cannot proceede further. As JniContext is not available at
+             * this point, we have to obtain exception details from scratch. This is not critical from performance
+             * perspective because missing handler usually denotes fatal condition.
+             *
+             * @param env JNI environment.
+             */
+            int ThrowOnMissingHandler(JNIEnv* env)
+            {
+                jclass cls = env->FindClass(C_PLATFORM_NO_CALLBACK_EXCEPTION);
+
+                env->ThrowNew(cls, "Callback handler is not set in native platform.");
+
+                return 0;
+            }
+
+            char* StringToChars(JNIEnv* env, jstring str, int* len) {
+                if (!str) {
+                    *len = 0;
+                    return NULL;
+                }
+
+                const char* strChars = env->GetStringUTFChars(str, 0);
+                const int strCharsLen = env->GetStringUTFLength(str);
+
+                char* strChars0 = new char[strCharsLen + 1];
+                std::strcpy(strChars0, strChars);
+                *(strChars0 + strCharsLen) = 0;
+
+                env->ReleaseStringUTFChars(str, strChars);
+
+                if (len)
+                    *len = strCharsLen;
+
+                return strChars0;
+            }
+
+            std::string JavaStringToCString(JNIEnv* env, jstring str, int* len)
+            {
+                char* resChars = StringToChars(env, str, len);
+
+                if (resChars)
+                {
+                    std::string res = std::string(resChars, *len);
+
+                    delete[] resChars;
+                    
+                    return res;
+                }
+                else
+                    return std::string();
+            }
+
+            jclass FindClass(JNIEnv* env, const char *name) {
+                jclass res = env->FindClass(name);
+
+                if (!res)
+                    throw JvmException();
+
+                jclass res0 = static_cast<jclass>(env->NewGlobalRef(res));
+
+                env->DeleteLocalRef(res);
+
+                return res0;
+            }
+
+            void DeleteClass(JNIEnv* env, jclass cls) {
+                if (cls)
+                    env->DeleteGlobalRef(cls);
+            }
+
+            void CheckClass(JNIEnv* env, const char *name)
+            {
+                jclass res = env->FindClass(name);
+
+                if (!res)
+                    throw JvmException();
+            }
+
+            jmethodID FindMethod(JNIEnv* env, jclass cls, JniMethod mthd) {
+                jmethodID mthd0 = mthd.isStatic ?
+                    env->GetStaticMethodID(cls, mthd.name, mthd.sign) : env->GetMethodID(cls, mthd.name, mthd.sign);
+
+                if (!mthd0)
+                    throw JvmException();
+
+                return mthd0;
+            }
+
+            void AddNativeMethod(JNINativeMethod* mthd, JniMethod jniMthd, void* fnPtr) {
+                mthd->name = jniMthd.name;
+                mthd->signature = jniMthd.sign;
+                mthd->fnPtr = fnPtr;
+            }
+
+            void JniJavaMembers::Initialize(JNIEnv* env) {
+                c_Class = FindClass(env, C_CLASS);
+                m_Class_getName = FindMethod(env, c_Class, M_CLASS_GET_NAME);
+
+                c_Throwable = FindClass(env, C_THROWABLE);
+                m_Throwable_getMessage = FindMethod(env, c_Throwable, M_THROWABLE_GET_MESSAGE);
+                m_Throwable_printStackTrace = FindMethod(env, c_Throwable, M_THROWABLE_PRINT_STACK_TRACE);
+            }
+
+            void JniJavaMembers::Destroy(JNIEnv* env) {
+                DeleteClass(env, c_Class);
+                DeleteClass(env, c_Throwable);
+            }
+
+            bool JniJavaMembers::WriteErrorInfo(JNIEnv* env, char** errClsName, int* errClsNameLen, char** errMsg, int* errMsgLen) {
+                if (env && env->ExceptionCheck()) {
+                    if (m_Class_getName && m_Throwable_getMessage) {
+                        jthrowable err = env->ExceptionOccurred();
+
+                        env->ExceptionClear();
+
+                        jclass errCls = env->GetObjectClass(err);
+
+                        jstring clsName = static_cast<jstring>(env->CallObjectMethod(errCls, m_Class_getName));
+                        *errClsName = StringToChars(env, clsName, errClsNameLen);
+
+                        jstring msg = static_cast<jstring>(env->CallObjectMethod(err, m_Throwable_getMessage));
+                        *errMsg = StringToChars(env, msg, errMsgLen);
+
+                        if (errCls)
+                            env->DeleteLocalRef(errCls);
+
+                        if (clsName)
+                            env->DeleteLocalRef(clsName);
+
+                        if (msg)
+                            env->DeleteLocalRef(msg);
+
+                        return true;
+                    }
+                    else {
+                        env->ExceptionClear();
+                    }
+                }
+
+                return false;
+            }
+
+            void JniMembers::Initialize(JNIEnv* env) {
+                c_PlatformAbstractQryCursor = FindClass(env, C_PLATFORM_ABSTRACT_QRY_CURSOR);
+                m_PlatformAbstractQryCursor_iter = FindMethod(env, c_PlatformAbstractQryCursor, M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER);
+                m_PlatformAbstractQryCursor_iterHasNext = FindMethod(env, c_PlatformAbstractQryCursor, M_PLATFORM_ABSTRACT_QRY_CURSOR_ITER_HAS_NEXT);
+                m_PlatformAbstractQryCursor_close = FindMethod(env, c_PlatformAbstractQryCursor, M_PLATFORM_ABSTRACT_QRY_CURSOR_CLOSE);
+
+                c_PlatformAffinity = FindClass(env, C_PLATFORM_AFFINITY);
+                m_PlatformAffinity_partitions = FindMethod(env, c_PlatformAffinity, C_PLATFORM_AFFINITY_PARTITIONS);
+
+                c_PlatformCache = FindClass(env, C_PLATFORM_CACHE);
+                m_PlatformCache_withSkipStore = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_SKIP_STORE);
+                m_PlatformCache_withNoRetries = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_NO_RETRIES);
+                m_PlatformCache_withExpiryPolicy = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_EXPIRY_PLC);
+                m_PlatformCache_withAsync = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_ASYNC);
+                m_PlatformCache_withKeepPortable = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_WITH_KEEP_PORTABLE);
+                m_PlatformCache_clear = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_CLEAR);
+                m_PlatformCache_removeAll = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_REMOVE_ALL);
+                m_PlatformCache_iterator = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_ITERATOR);
+                m_PlatformCache_localIterator = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_LOCAL_ITERATOR);
+                m_PlatformCache_enterLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_ENTER_LOCK);
+                m_PlatformCache_exitLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_EXIT_LOCK);
+                m_PlatformCache_tryEnterLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_TRY_ENTER_LOCK);
+                m_PlatformCache_closeLock = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_CLOSE_LOCK);
+                m_PlatformCache_rebalance = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_REBALANCE);
+                m_PlatformCache_size = FindMethod(env, c_PlatformCache, M_PLATFORM_CACHE_SIZE);
+
+                c_PlatformCacheStoreCallback = FindClass(env, C_PLATFORM_CACHE_STORE_CALLBACK);
+                m_PlatformCacheStoreCallback_invoke = FindMethod(env, c_PlatformCacheStoreCallback, M_PLATFORM_CACHE_STORE_CALLBACK_INVOKE);
+
+                c_IgniteException = FindClass(env, C_IGNITE_EXCEPTION);
+
+                c_PlatformClusterGroup = FindClass(env, C_PLATFORM_CLUSTER_GRP);
+                m_PlatformClusterGroup_forOthers = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_OTHERS);
+                m_PlatformClusterGroup_forRemotes = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_REMOTES);
+                m_PlatformClusterGroup_forDaemons = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_DAEMONS);
+                m_PlatformClusterGroup_forRandom = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_RANDOM);
+                m_PlatformClusterGroup_forOldest = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_OLDEST);
+                m_PlatformClusterGroup_forYoungest = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_FOR_YOUNGEST);
+                m_PlatformClusterGroup_resetMetrics = FindMethod(env, c_PlatformClusterGroup, M_PLATFORM_CLUSTER_GRP_RESET_METRICS);
+
+                c_PlatformCompute = FindClass(env, C_PLATFORM_COMPUTE);
+                m_PlatformCompute_withNoFailover = FindMethod(env, c_PlatformCompute, M_PLATFORM_COMPUTE_WITH_NO_FAILOVER);
+                m_PlatformCompute_withTimeout = FindMethod(env, c_PlatformCompute, M_PLATFORM_COMPUTE_WITH_TIMEOUT);
+                m_PlatformCompute_executeNative = FindMethod(env, c_PlatformCompute, M_PLATFORM_COMPUTE_EXECUTE_NATIVE);
+
+                c_PlatformContinuousQuery = FindClass(env, C_PLATFORM_CONT_QRY);
+                m_PlatformContinuousQuery_close = FindMethod(env, c_PlatformContinuousQuery, M_PLATFORM_CONT_QRY_CLOSE);
+                m_PlatformContinuousQuery_getInitialQueryCursor = FindMethod(env, c_PlatformContinuousQuery, M_PLATFORM_CONT_QRY_GET_INITIAL_QUERY_CURSOR);
+
+                c_PlatformDataStreamer = FindClass(env, C_PLATFORM_DATA_STREAMER);
+                m_PlatformDataStreamer_listenTopology = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_LISTEN_TOPOLOGY);
+                m_PlatformDataStreamer_getAllowOverwrite = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_ALLOW_OVERWRITE);
+                m_PlatformDataStreamer_setAllowOverwrite = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_ALLOW_OVERWRITE);
+                m_PlatformDataStreamer_getSkipStore = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_SKIP_STORE);
+                m_PlatformDataStreamer_setSkipStore = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_SKIP_STORE);
+                m_PlatformDataStreamer_getPerNodeBufSize = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_PER_NODE_BUFFER_SIZE);
+                m_PlatformDataStreamer_setPerNodeBufSize = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_PER_NODE_BUFFER_SIZE);
+                m_PlatformDataStreamer_getPerNodeParallelOps = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_GET_PER_NODE_PARALLEL_OPS);
+                m_PlatformDataStreamer_setPerNodeParallelOps = FindMethod(env, c_PlatformDataStreamer, M_PLATFORM_DATA_STREAMER_SET_PER_NODE_PARALLEL_OPS);
+                
+                c_PlatformEvents = FindClass(env, C_PLATFORM_EVENTS);
+                m_PlatformEvents_withAsync = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_WITH_ASYNC);
+                m_PlatformEvents_stopLocalListen = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_STOP_LOCAL_LISTEN);
+                m_PlatformEvents_localListen = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_LOCAL_LISTEN);
+                m_PlatformEvents_isEnabled = FindMethod(env, c_PlatformEvents, M_PLATFORM_EVENTS_IS_ENABLED);
+                
+				c_PlatformServices = FindClass(env, C_PLATFORM_SERVICES);
+				m_PlatformServices_withAsync = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_WITH_ASYNC);
+				m_PlatformServices_withServerKeepPortable = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_WITH_SERVER_KEEP_PORTABLE);
+				m_PlatformServices_cancel = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_CANCEL);
+				m_PlatformServices_cancelAll = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_CANCEL_ALL);
+				m_PlatformServices_serviceProxy = FindMethod(env, c_PlatformServices, M_PLATFORM_SERVICES_SERVICE_PROXY);
+
+                c_PlatformIgnition = FindClass(env, C_PLATFORM_IGNITION);
+                m_PlatformIgnition_start = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_START);
+                m_PlatformIgnition_instance = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_INSTANCE);
+                m_PlatformIgnition_environmentPointer = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_ENVIRONMENT_POINTER);
+                m_PlatformIgnition_stop = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_STOP);
+                m_PlatformIgnition_stopAll = FindMethod(env, c_PlatformIgnition, M_PLATFORM_IGNITION_STOP_ALL);
+
+                c_PlatformMessaging = FindClass(env, C_PLATFORM_MESSAGING);
+                m_PlatformMessaging_withAsync = FindMethod(env, c_PlatformMessaging, M_PLATFORM_MESSAGING_WITH_ASYNC);
+
+                c_PlatformProcessor = FindClass(env, C_PLATFORM_PROCESSOR);
+                m_PlatformProcessor_releaseStart = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_RELEASE_START);
+                m_PlatformProcessor_cache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_CACHE);
+                m_PlatformProcessor_createCache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_CREATE_CACHE);
+                m_PlatformProcessor_getOrCreateCache = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE);
+                m_PlatformProcessor_affinity = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_AFFINITY);
+                m_PlatformProcessor_dataStreamer = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_DATA_STREAMER);
+                m_PlatformProcessor_transactions = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_TRANSACTIONS);
+                m_PlatformProcessor_projection = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_PROJECTION);
+                m_PlatformProcessor_compute = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_COMPUTE);
+                m_PlatformProcessor_message = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_MESSAGE);
+                m_PlatformProcessor_events = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_EVENTS);
+                m_PlatformProcessor_services = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_SERVICES);
+                m_PlatformProcessor_extensions = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_EXTENSIONS);
+
+                c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET);
+                m_PlatformTarget_inStreamOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_LONG);
+                m_PlatformTarget_inStreamOutObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT);
+                m_PlatformTarget_outLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_LONG);
+                m_PlatformTarget_outStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_STREAM);
+                m_PlatformTarget_outObject = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_OUT_OBJECT);
+                m_PlatformTarget_inStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_STREAM_OUT_STREAM);
+                m_PlatformTarget_inObjectStreamOutStream = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_OBJECT_STREAM_OUT_STREAM);
+                m_PlatformTarget_listenFuture = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FUTURE);
+                m_PlatformTarget_listenFutureForOperation = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_LISTEN_FOR_OPERATION);
+
+                c_PlatformTransactions = FindClass(env, C_PLATFORM_TRANSACTIONS);
+                m_PlatformTransactions_txStart = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_START);
+                m_PlatformTransactions_txCommit = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_COMMIT);
+                m_PlatformTransactions_txRollback = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_ROLLBACK);
+                m_PlatformTransactions_txCommitAsync = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_COMMIT_ASYNC);
+                m_PlatformTransactions_txRollbackAsync = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_ROLLBACK_ASYNC);
+                m_PlatformTransactions_txState = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_STATE);
+                m_PlatformTransactions_txSetRollbackOnly = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_SET_ROLLBACK_ONLY);
+                m_PlatformTransactions_txClose = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_TX_CLOSE);
+                m_PlatformTransactions_resetMetrics = FindMethod(env, c_PlatformTransactions, M_PLATFORM_TRANSACTIONS_RESET_METRICS);
+
+                c_PlatformUtils = FindClass(env, C_PLATFORM_UTILS);
+                m_PlatformUtils_reallocate = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_REALLOC);
+                m_PlatformUtils_errData = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_ERR_DATA);
+
+                // Find utility classes which are not used from context, but are still required in other places.
+                CheckClass(env, C_PLATFORM_NO_CALLBACK_EXCEPTION);
+            }
+
+            void JniMembers::Destroy(JNIEnv* env) {
+                DeleteClass(env, c_PlatformAbstractQryCursor);
+                DeleteClass(env, c_PlatformAffinity);
+                DeleteClass(env, c_PlatformCache);
+                DeleteClass(env, c_PlatformCacheStoreCallback);
+                DeleteClass(env, c_IgniteException);
+                DeleteClass(env, c_PlatformClusterGroup);
+                DeleteClass(env, c_PlatformCompute);
+                DeleteClass(env, c_PlatformContinuousQuery);
+                DeleteClass(env, c_PlatformDataStreamer);
+                DeleteClass(env, c_PlatformEvents);
+                DeleteClass(env, c_PlatformIgnition);
+                DeleteClass(env, c_PlatformMessaging);
+                DeleteClass(env, c_PlatformProcessor);
+                DeleteClass(env, c_PlatformTarget);
+                DeleteClass(env, c_PlatformTransactions);
+                DeleteClass(env, c_PlatformUtils);
+            }
+
+            JniJvm::JniJvm() : jvm(NULL), javaMembers(JniJavaMembers()), members(JniMembers())
+            {
+                // No-op.
+            }
+
+            JniJvm::JniJvm(JavaVM* jvm, JniJavaMembers javaMembers, JniMembers members) : 
+                jvm(jvm), javaMembers(javaMembers), members(members)
+            {
+                // No-op.
+            }
+
+            JavaVM* JniJvm::GetJvm()
+            {
+                return jvm;
+            }
+
+            JniJavaMembers& JniJvm::GetJavaMembers()
+            {
+                return javaMembers;
+            }
+
+            JniMembers& JniJvm::GetMembers()
+            {
+                return members;
+            }
+
+            /*
+             * Create JVM.
+             */
+            void CreateJvm(char** opts, int optsLen, JavaVM** jvm, JNIEnv** env) {
+                JavaVMOption* opts0 = new JavaVMOption[optsLen];
+
+                for (int i = 0; i < optsLen; i++)
+                    opts0[i].optionString = *(opts + i);
+
+                JavaVMInitArgs args;
+
+                args.version = JNI_VERSION_1_6;
+                args.nOptions = optsLen;
+                args.options = opts0;
+                args.ignoreUnrecognized = 0;
+
+                jint res = JNI_CreateJavaVM(jvm, reinterpret_cast<void**>(env), &args);
+
+                delete[] opts0;
+
+                if (res != JNI_OK)
+                    throw JvmException();
+            }
+
+            void RegisterNatives(JNIEnv* env) {
+                {
+					JNINativeMethod methods[52];
+
+                    int idx = 0;
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_CREATE, reinterpret_cast<void*>(JniCacheStoreCreate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_INVOKE, reinterpret_cast<void*>(JniCacheStoreInvoke));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_DESTROY, reinterpret_cast<void*>(JniCacheStoreDestroy));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_STORE_SESSION_CREATE, reinterpret_cast<void*>(JniCacheStoreSessionCreate));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_ENTRY_FILTER_CREATE, reinterpret_cast<void*>(JniCacheEntryFilterCreate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_ENTRY_FILTER_APPLY, reinterpret_cast<void*>(JniCacheEntryFilterApply));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_ENTRY_FILTER_DESTROY, reinterpret_cast<void*>(JniCacheEntryFilterDestroy));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CACHE_INVOKE, reinterpret_cast<void*>(JniCacheInvoke));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_MAP, reinterpret_cast<void*>(JniComputeTaskMap));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_JOB_RESULT, reinterpret_cast<void*>(JniComputeTaskJobResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_REDUCE, reinterpret_cast<void*>(JniComputeTaskReduce));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_TASK_COMPLETE, reinterpret_cast<void*>(JniComputeTaskComplete));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_SERIALIZE, reinterpret_cast<void*>(JniComputeJobSerialize));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_CREATE, reinterpret_cast<void*>(JniComputeJobCreate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_EXECUTE, reinterpret_cast<void*>(JniComputeJobExecute));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_DESTROY, reinterpret_cast<void*>(JniComputeJobDestroy));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_COMPUTE_JOB_CANCEL, reinterpret_cast<void*>(JniComputeJobCancel));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_LSNR_APPLY, reinterpret_cast<void*>(JniContinuousQueryListenerApply));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_CREATE, reinterpret_cast<void*>(JniContinuousQueryFilterCreate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_EVAL, reinterpret_cast<void*>(JniContinuousQueryFilterApply));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_RELEASE, reinterpret_cast<void*>(JniContinuousQueryFilterRelease));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_TOPOLOGY_UPDATE, reinterpret_cast<void*>(JniDataStreamerTopologyUpdate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_STREAM_RECEIVER_INVOKE, reinterpret_cast<void*>(JniDataStreamerStreamReceiverInvoke));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_BYTE_RES, reinterpret_cast<void*>(JniFutureByteResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_BOOL_RES, reinterpret_cast<void*>(JniFutureBoolResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_SHORT_RES, reinterpret_cast<void*>(JniFutureShortResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_CHAR_RES, reinterpret_cast<void*>(JniFutureCharResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_INT_RES, reinterpret_cast<void*>(JniFutureIntResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_FLOAT_RES, reinterpret_cast<void*>(JniFutureFloatResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_LONG_RES, reinterpret_cast<void*>(JniFutureLongResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_DOUBLE_RES, reinterpret_cast<void*>(JniFutureDoubleResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_OBJ_RES, reinterpret_cast<void*>(JniFutureObjectResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_NULL_RES, reinterpret_cast<void*>(JniFutureNullResult));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_FUTURE_ERR, reinterpret_cast<void*>(JniFutureError));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_LIFECYCLE_EVENT, reinterpret_cast<void*>(JniLifecycleEvent));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_MESSAGING_FILTER_CREATE, reinterpret_cast<void*>(JniMessagingFilterCreate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_MESSAGING_FILTER_APPLY, reinterpret_cast<void*>(JniMessagingFilterApply));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_MESSAGING_FILTER_DESTROY, reinterpret_cast<void*>(JniMessagingFilterDestroy));
+                    
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EVENT_FILTER_CREATE, reinterpret_cast<void*>(JniEventFilterCreate));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EVENT_FILTER_APPLY, reinterpret_cast<void*>(JniEventFilterApply));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EVENT_FILTER_DESTROY, reinterpret_cast<void*>(JniEventFilterDestroy));
+                    
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_SERVICE_INIT, reinterpret_cast<void*>(JniServiceInit));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_SERVICE_EXECUTE, reinterpret_cast<void*>(JniServiceExecute));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_SERVICE_CANCEL, reinterpret_cast<void*>(JniServiceCancel));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_SERVICE_INVOKE_METHOD, reinterpret_cast<void*>(JniServiceInvokeMethod));
+					
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_CLUSTER_NODE_FILTER_APPLY, reinterpret_cast<void*>(JniClusterNodeFilterApply));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_NODE_INFO, reinterpret_cast<void*>(JniNodeInfo));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_MEMORY_REALLOCATE, reinterpret_cast<void*>(JniMemoryReallocate));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_ON_START, reinterpret_cast<void*>(JniOnStart));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_ON_STOP, reinterpret_cast<void*>(JniOnStop));
+
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_OUT_LONG, reinterpret_cast<void*>(JniExtensionCallbackInLongOutLong));
+                    AddNativeMethod(methods + idx++, M_PLATFORM_CALLBACK_UTILS_EXTENSION_CALLBACK_IN_LONG_LONG_OUT_LONG, reinterpret_cast<void*>(JniExtensionCallbackInLongLongOutLong));
+
+                    jint res = env->RegisterNatives(FindClass(env, C_PLATFORM_CALLBACK_UTILS), methods, idx);
+
+                    if (res != JNI_OK)
+                        throw JvmException();
+                }  
+            }
+
+            JniContext::JniContext(JniJvm* jvm, JniHandlers hnds) : jvm(jvm), hnds(hnds) {
+                // No-op.
+            }
+
+            JniContext* JniContext::Create(char** opts, int optsLen, JniHandlers hnds) {
+                return Create(opts, optsLen, hnds, NULL);
+            }
+
+            JniContext* JniContext::Create(char** opts, int optsLen, JniHandlers hnds, JniErrorInfo* errInfo)
+            {
+                // Acquire global lock to instantiate the JVM.
+                JVM_LOCK.Enter();
+
+                // Define local variables.
+                JavaVM* jvm = NULL;
+                JNIEnv* env = NULL;
+
+                JniJavaMembers javaMembers;
+                memset(&javaMembers, 0, sizeof(javaMembers));
+
+                JniMembers members;
+                memset(&members, 0, sizeof(members));
+
+                JniContext* ctx = NULL;
+
+                std::string errClsName;
+                int errClsNameLen = 0;
+                std::string errMsg;
+                int errMsgLen = 0;
+
+                try {
+                    if (!JVM.GetJvm()) {
+                        // 1. Create JVM itself.    
+                        CreateJvm(opts, optsLen, &jvm, &env);
+
+                        // 2. Populate members;
+                        javaMembers.Initialize(env);
+                        members.Initialize(env);
+
+                        // 3. Register native functions.
+                        RegisterNatives(env);
+
+                        // 4. Create JNI JVM.
+                        JVM = JniJvm(jvm, javaMembers, members);
+
+                        char* printStack = getenv("IGNITE_CPP_PRINT_STACK");
+                        PRINT_EXCEPTION = printStack && strcmp("true", printStack) == 0;
+                    }
+
+                    ctx = new JniContext(&JVM, hnds);
+                }
+                catch (JvmException)
+                {
+                    char* errClsNameChars = NULL;
+                    char* errMsgChars = NULL;
+
+                    // Read error info if possible.
+                    javaMembers.WriteErrorInfo(env, &errClsNameChars, &errClsNameLen, &errMsgChars, &errMsgLen);
+
+                    if (errClsNameChars) {
+                        errClsName = errClsNameChars;
+
+                        delete[] errClsNameChars;
+                    }
+
+                    if (errMsgChars)
+                    {
+                        errMsg = errMsgChars;
+
+                        delete[] errMsgChars;
+                    }
+
+                    // Destroy mmebers.
+                    if (env) {
+                        members.Destroy(env);
+                        javaMembers.Destroy(env);
+                    }
+
+                    // Destroy faulty JVM.
+                    if (jvm)
+                        jvm->DestroyJavaVM();
+                }
+
+                // It safe to release the lock at this point.
+                JVM_LOCK.Leave();
+
+                // Notify err callback if needed.
+                if (!ctx) {
+                    if (errInfo) {
+                        JniErrorInfo errInfo0(IGNITE_JNI_ERR_JVM_INIT, errClsName.c_str(), errMsg.c_str());
+
+                        *errInfo = errInfo0;
+                    }
+
+                    if (hnds.error)
+                        hnds.error(hnds.target, IGNITE_JNI_ERR_JVM_INIT, errClsName.c_str(), errClsNameLen,
+                            errMsg.c_str(), errMsgLen, NULL, 0);
+                }
+
+                return ctx;
+            }
+
+            int JniContext::Reallocate(long long memPtr, int cap) {
+                JavaVM* jvm = JVM.GetJvm();
+
+                JNIEnv* env;
+
+                int attachRes = jvm->AttachCurrentThread(reinterpret_cast<void**>(&env), NULL);
+
+                if (attachRes == JNI_OK)
+                    AttachHelper::OnThreadAttach();
+                else
+                    return -1;
+
+                env->CallStaticVoidMethod(JVM.GetMembers().c_PlatformUtils, JVM.GetMembers().m_PlatformUtils_reallocate, memPtr, cap);
+
+                if (env->ExceptionCheck()) {
+                    env->ExceptionClear();
+
+                    return -1;
+                }
+
+                return 0;
+            }
+
+            void JniContext::Detach() {
+                gcc::Memory::Fence();
+
+                if (JVM.GetJvm()) {
+                    JNIEnv* env;
+
+                    JVM.GetJvm()->GetEnv(reinterpret_cast<void**>(&env), JNI_VERSION_1_6);
+
+                    if (env)
+                        JVM.GetJvm()->DetachCurrentThread();
+                }
+            }
+
+            jobject JniContext::IgnitionStart(char* cfgPath, char* gridName, int factoryId, long long dataPtr) {
+                return IgnitionStart(cfgPath, gridName, factoryId, dataPtr, NULL);
+            }
+            
+            jobject JniContext::IgnitionStart(char* cfgPath, char* gridName, int factoryId, long long dataPtr, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jstring cfgPath0 = env->NewStringUTF(cfgPath);
+                jstring gridName0 = env->NewStringUTF(gridName);
+
+                jobject interop = env->CallStaticObjectMethod(
+                    jvm->GetMembers().c_PlatformIgnition,
+                    jvm->GetMembers().m_PlatformIgnition_start,
+                    cfgPath0,
+                    gridName0,
+                    factoryId,
+                    reinterpret_cast<long long>(&hnds),
+                    dataPtr
+                );
+
+                ExceptionCheck(env, errInfo);
+
+                return LocalToGlobal(env, interop);
+            }
+
+
+            jobject JniContext::IgnitionInstance(char* gridName)
+            {
+                return IgnitionInstance(gridName, NULL);
+            }
+
+            jobject JniContext::IgnitionInstance(char* gridName, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jstring gridName0 = env->NewStringUTF(gridName);
+
+                jobject interop = env->CallStaticObjectMethod(jvm->GetMembers().c_PlatformIgnition,
+                    jvm->GetMembers().m_PlatformIgnition_instance, gridName0);
+
+                ExceptionCheck(env, errInfo);
+
+                return LocalToGlobal(env, interop);
+            }
+
+            long long JniContext::IgnitionEnvironmentPointer(char* gridName)
+            {
+                return IgnitionEnvironmentPointer(gridName, NULL);
+            }
+
+            long long JniContext::IgnitionEnvironmentPointer(char* gridName, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jstring gridName0 = env->NewStringUTF(gridName);
+
+                long long res = env->CallStaticLongMethod(jvm->GetMembers().c_PlatformIgnition,
+                    jvm->GetMembers().m_PlatformIgnition_environmentPointer, gridName0);
+
+                ExceptionCheck(env, errInfo);
+
+                return res;
+            }
+
+            bool JniContext::IgnitionStop(char* gridName, bool cancel)
+            {
+                return IgnitionStop(gridName, cancel, NULL);
+            }
+
+            bool JniContext::IgnitionStop(char* gridName, bool cancel, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jstring gridName0 = env->NewStringUTF(gridName);
+
+                jboolean res = env->CallStaticBooleanMethod(jvm->GetMembers().c_PlatformIgnition, jvm->GetMembers().m_PlatformIgnition_stop,
+                    gridName0, cancel);
+
+                ExceptionCheck(env, errInfo);
+
+                return res != 0;
+            }
+
+            void JniContext::IgnitionStopAll(bool cancel)
+            {
+                return IgnitionStopAll(cancel, NULL);
+            }
+
+            void JniContext::IgnitionStopAll(bool cancel, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                env->CallStaticVoidMethod(jvm->GetMembers().c_PlatformIgnition, jvm->GetMembers().m_PlatformIgnition_stopAll, cancel);
+
+                ExceptionCheck(env, errInfo);
+            }
+
+            void JniContext::ProcessorReleaseStart(jobject obj) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformProcessor_releaseStart);
+
+                ExceptionCheck(env);
+            }
+
+            jobject JniContext::ProcessorProjection(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject prj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_projection);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, prj);
+            }
+
+            jobject JniContext::ProcessorCache0(jobject obj, const char* name, jmethodID mthd, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL;
+
+                jobject cache = env->CallObjectMethod(obj, mthd, name0);
+
+                if (name0)
+                    env->DeleteLocalRef(name0);
+
+                ExceptionCheck(env, errInfo);
+
+                return LocalToGlobal(env, cache);
+            }
+
+            jobject JniContext::ProcessorCache(jobject obj, const char* name) {
+                return ProcessorCache(obj, name, NULL);
+            }
+
+            jobject JniContext::ProcessorCache(jobject obj, const char* name, JniErrorInfo* errInfo) {
+                return ProcessorCache0(obj, name, jvm->GetMembers().m_PlatformProcessor_cache, errInfo);
+            }
+
+            jobject JniContext::ProcessorCreateCache(jobject obj, const char* name) {
+                return ProcessorCreateCache(obj, name, NULL);
+            }
+
+            jobject JniContext::ProcessorCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo)
+            {
+                return ProcessorCache0(obj, name, jvm->GetMembers().m_PlatformProcessor_createCache, errInfo);
+            }
+
+            jobject JniContext::ProcessorGetOrCreateCache(jobject obj, const char* name) {
+                return ProcessorGetOrCreateCache(obj, name, NULL);
+            }
+
+            jobject JniContext::ProcessorGetOrCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo)
+            {
+                return ProcessorCache0(obj, name, jvm->GetMembers().m_PlatformProcessor_getOrCreateCache, errInfo);
+            }
+
+            jobject JniContext::ProcessorAffinity(jobject obj, const char* name) {
+                JNIEnv* env = Attach();
+
+                jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL;
+
+                jobject aff = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_affinity, name0);
+
+                if (name0)
+                    env->DeleteLocalRef(name0);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, aff);
+            }
+
+            jobject JniContext::ProcessorDataStreamer(jobject obj, const char* name, bool keepPortable) {
+                JNIEnv* env = Attach();
+
+                jstring name0 = name != NULL ? env->NewStringUTF(name) : NULL;
+
+                jobject ldr = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_dataStreamer, name0,
+                    keepPortable);
+
+                if (name0)
+                    env->DeleteLocalRef(name0);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, ldr);
+            }
+
+            jobject JniContext::ProcessorTransactions(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject tx = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_transactions);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, tx);
+            }
+            
+            jobject JniContext::ProcessorCompute(jobject obj, jobject prj) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_compute, prj);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::ProcessorMessage(jobject obj, jobject prj) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_message, prj);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::ProcessorEvents(jobject obj, jobject prj) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_events, prj);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::ProcessorServices(jobject obj, jobject prj) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_services, prj);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+            
+            jobject JniContext::ProcessorExtensions(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_extensions);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                long long res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformTarget_inStreamOutLong, opType, memPtr);
+
+                ExceptionCheck(env, err);
+
+                return res;
+            }
+
+            void JniContext::TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_inStreamOutStream, opType, inMemPtr, outMemPtr);
+
+                ExceptionCheck(env, err);
+            }
+
+           jobject JniContext::TargetInStreamOutObject(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, opType, memPtr);
+
+                ExceptionCheck(env, err);
+
+                return LocalToGlobal(env, res);
+            }
+
+            void JniContext::TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_inObjectStreamOutStream, opType, arg, inMemPtr, outMemPtr);
+
+                ExceptionCheck(env, err);
+            }
+
+            long long JniContext::TargetOutLong(jobject obj, int opType, JniErrorInfo* err)
+            {
+                JNIEnv* env = Attach();
+
+                jlong res = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformTarget_outLong, opType);
+
+                ExceptionCheck(env, err);
+
+                return res;
+            }
+
+            void JniContext::TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_outStream, opType, memPtr);
+
+                ExceptionCheck(env, err);
+            }
+
+            jobject JniContext::TargetOutObject(jobject obj, int opType, JniErrorInfo* err)
+            {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformTarget_outObject, opType);
+
+                ExceptionCheck(env, err);
+
+                return LocalToGlobal(env, res);
+            }
+
+            void JniContext::TargetListenFuture(jobject obj, long long futId, int typ) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_listenFuture, futId, typ);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::TargetListenFutureForOperation(jobject obj, long long futId, int typ, int opId) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTarget_listenFutureForOperation, futId, typ, opId);
+
+                ExceptionCheck(env);
+            }
+
+            int JniContext::AffinityPartitions(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jint parts = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformAffinity_partitions);
+
+                ExceptionCheck(env);
+
+                return parts;
+            }
+
+            jobject JniContext::CacheWithSkipStore(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withSkipStore);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, cache);
+            }
+
+            jobject JniContext::CacheWithNoRetries(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withNoRetries);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, cache);
+            }
+
+            jobject JniContext::CacheWithExpiryPolicy(jobject obj, long long create, long long update, long long access) {
+                JNIEnv* env = Attach();
+
+                jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withExpiryPolicy,
+                    create, update, access);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, cache);
+            }
+
+            jobject JniContext::CacheWithAsync(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withAsync);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, cache);
+            }
+
+            jobject JniContext::CacheWithKeepPortable(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject cache = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_withKeepPortable);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, cache);
+            }
+
+            void JniContext::CacheClear(jobject obj, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_clear);
+
+                ExceptionCheck(env, err);
+            }
+
+            void JniContext::CacheRemoveAll(jobject obj, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_removeAll);
+
+                ExceptionCheck(env, err);
+            }
+
+            jobject JniContext::CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(
+                    obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, type, memPtr);
+
+                ExceptionCheck(env, err);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(
+                    obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, type, memPtr);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::CacheIterator(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_iterator);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            jobject JniContext::CacheLocalIterator(jobject obj, int peekModes) {
+                JNIEnv*env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformCache_localIterator, peekModes);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+            void JniContext::CacheEnterLock(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_enterLock, id);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::CacheExitLock(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_exitLock, id);
+
+                ExceptionCheck(env);
+            }
+
+            bool JniContext::CacheTryEnterLock(jobject obj, long long id, long long timeout) {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformCache_tryEnterLock, id, timeout);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::CacheCloseLock(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_closeLock, id);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::CacheRebalance(jobject obj, long long futId) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCache_rebalance, futId);
+
+                ExceptionCheck(env);
+            }
+
+            int JniContext::CacheSize(jobject obj, int peekModes, bool loc, JniErrorInfo* err) {
+                JNIEnv* env = Attach();
+
+                jint res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformCache_size, peekModes, loc);
+
+                ExceptionCheck(env, err);
+
+                return res;
+            }
+
+            void JniContext::CacheStoreCallbackInvoke(jobject obj, long long memPtr) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCacheStoreCallback_invoke, memPtr);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::ComputeWithNoFailover(jobject obj) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCompute_withNoFailover);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::ComputeWithTimeout(jobject obj, long long timeout) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCompute_withTimeout, timeout);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::ComputeExecuteNative(jobject obj, long long taskPtr, long long topVer) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformCompute_executeNative, taskPtr, topVer);
+
+                ExceptionCheck(env);
+            }
+
+            void JniContext::ContinuousQueryClose(jobject obj) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformContinuousQuery_close);
+
+                ExceptionCheck(env);
+            }
+
+            void* JniContext::ContinuousQueryGetInitialQueryCursor(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformContinuousQuery_getInitialQueryCursor);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            void JniContext::DataStreamerListenTopology(jobject obj, long long ptr) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_listenTopology, ptr);
+
+                ExceptionCheck(env);
+            }
+
+            bool JniContext::DataStreamerAllowOverwriteGet(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getAllowOverwrite);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::DataStreamerAllowOverwriteSet(jobject obj, bool val) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setAllowOverwrite, val);
+
+                ExceptionCheck(env);
+            }
+
+            bool JniContext::DataStreamerSkipStoreGet(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getSkipStore);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::DataStreamerSkipStoreSet(jobject obj, bool val) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setSkipStore, val);
+
+                ExceptionCheck(env);
+            }
+
+            int JniContext::DataStreamerPerNodeBufferSizeGet(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jint res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getPerNodeBufSize);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            void JniContext::DataStreamerPerNodeBufferSizeSet(jobject obj, int val) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setPerNodeBufSize, val);
+
+                ExceptionCheck(env);
+            }
+
+            int JniContext::DataStreamerPerNodeParallelOperationsGet(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jint res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_getPerNodeParallelOps);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            void JniContext::DataStreamerPerNodeParallelOperationsSet(jobject obj, int val) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformDataStreamer_setPerNodeParallelOps, val);
+
+                ExceptionCheck(env);
+            }
+
+            jobject JniContext::MessagingWithAsync(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject msg = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformMessaging_withAsync);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, msg);
+            }
+
+            jobject JniContext::ProjectionForOthers(jobject obj, jobject prj) {
+                JNIEnv* env = Attach();
+
+                jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forOthers, prj);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, newPrj);
+            }
+
+            jobject JniContext::ProjectionForRemotes(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forRemotes);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, newPrj);
+            }
+
+            jobject JniContext::ProjectionForDaemons(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forDaemons);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, newPrj);
+            }
+
+            jobject JniContext::ProjectionForRandom(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forRandom);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, newPrj);
+            }
+
+            jobject JniContext::ProjectionForOldest(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forOldest);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, newPrj);
+            }
+
+            jobject JniContext::ProjectionForYoungest(jobject obj) {
+                JNIEnv* env = Attach();
+
+                jobject newPrj = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_forYoungest);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, newPrj);
+            }
+
+            void JniContext::ProjectionResetMetrics(jobject obj) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformClusterGroup_resetMetrics);
+
+                ExceptionCheck(env);
+            }
+
+            jobject JniContext::ProjectionOutOpRet(jobject obj, int type, long long memPtr) {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(
+                    obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, type, memPtr);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
+
+            void JniContext::QueryCursorIterator(jobject obj, JniErrorInfo* errInfo) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAbstractQryCursor_iter);
+
+                ExceptionCheck(env, errInfo);
+            }
+
+            bool JniContext::QueryCursorIteratorHasNext(jobject obj, JniErrorInfo* errInfo)
+            {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformAbstractQryCursor_iterHasNext);
+
+                ExceptionCheck(env, errInfo);
+
+                return res != 0;
+            }
+
+            void JniContext::QueryCursorClose(jobject obj, JniErrorInfo* errInfo) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformAbstractQryCursor_close);
+
+                ExceptionCheck(env, errInfo);
+            }
+
+            long long JniContext::TransactionsStart(jobject obj, int concurrency, int isolation, long long timeout, int txSize) {
+                JNIEnv* env = Attach();
+
+                long long id = env->CallLongMethod(obj, jvm->GetMembers().m_PlatformTransactions_txStart, concurrency, isolation, timeout, txSize);
+
+                ExceptionCheck(env);
+
+                return id;
+            }
+
+            int JniContext::TransactionsCommit(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                int res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txCommit, id);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            void JniContext::TransactionsCommitAsync(jobject obj, long long id, long long futId) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTransactions_txCommitAsync, id, futId);
+
+                ExceptionCheck(env);
+            }
+
+            int JniContext::TransactionsRollback(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                int res = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txRollback, id);
+
+                ExceptionCheck(env);
+
+                return res;
+            }
+
+            void JniContext::TransactionsRollbackAsync(jobject obj, long long id, long long futId) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTransactions_txRollbackAsync, id, futId);
+
+                ExceptionCheck(env);
+            }
+
+            int JniContext::TransactionsClose(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                jint state = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txClose, id);
+
+                ExceptionCheck(env);
+
+                return state;
+            }
+
+            int JniContext::TransactionsState(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                jint state = env->CallIntMethod(obj, jvm->GetMembers().m_PlatformTransactions_txState, id);
+
+                ExceptionCheck(env);
+
+                return state;
+            }
+
+            bool JniContext::TransactionsSetRollbackOnly(jobject obj, long long id) {
+                JNIEnv* env = Attach();
+
+                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformTransactions_txSetRollbackOnly, id);
+
+                ExceptionCheck(env);
+
+                return res != 0;
+            }
+
+            void JniContext::TransactionsResetMetrics(jobject obj) {
+                JNIEnv* env = Attach();
+
+                env->CallVoidMethod(obj, jvm->GetMembers().m_PlatformTransactions_resetMetrics);
+
+                ExceptionCheck(env);
+            }
+
+            jobject JniContext::EventsWithAsync(jobject obj) {
+                JNIEnv * env = Attach();
+
+                jobject res = env->CallObjectMethod(

<TRUNCATED>

[13/29] ignite git commit: Platforms: reworked PlatformCacheEntryFilter interface.

Posted by ak...@apache.org.
Platforms: reworked PlatformCacheEntryFilter interface.


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

Branch: refs/heads/ignite-843
Commit: 6d0aff43c4e30069cab19a8c71c96b4e47b9ae8d
Parents: 66d46ec
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 11:29:55 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 11:29:55 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  5 +-
 .../cache/GridLoadCacheCloseablePredicate.java  | 30 -----------
 .../distributed/dht/GridDhtCacheAdapter.java    |  6 +--
 .../CacheQueryCloseableScanBiPredicate.java     | 30 -----------
 .../cache/query/GridCacheQueryManager.java      | 52 ++++++++++----------
 .../cache/PlatformCacheEntryFilter.java         | 11 +++--
 6 files changed, 39 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6d0aff43/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 653c2b1..68749b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -94,6 +94,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
@@ -3470,8 +3471,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         }
         finally {
-            if (p instanceof GridLoadCacheCloseablePredicate)
-                ((GridLoadCacheCloseablePredicate)p).onClose();
+            if (p instanceof PlatformCacheEntryFilter)
+                ((PlatformCacheEntryFilter)p).onClose();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d0aff43/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLoadCacheCloseablePredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLoadCacheCloseablePredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLoadCacheCloseablePredicate.java
deleted file mode 100644
index 1331a42..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLoadCacheCloseablePredicate.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.lang.*;
-
-/**
- * Special version of bi-predicate for LoadCache with close callback.
- */
-public interface GridLoadCacheCloseablePredicate<K, V> extends IgniteBiPredicate<K, V> {
-    /**
-     * Callback invoked when predicate is no longer needed.
-     */
-    public void onClose();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d0aff43/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index ecacb29..ee56b16 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -50,7 +50,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedExceptio
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory;
 import org.apache.ignite.internal.processors.cache.GridCachePreloader;
-import org.apache.ignite.internal.processors.cache.GridLoadCacheCloseablePredicate;
 import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest;
@@ -62,6 +61,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCach
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridIteratorAdapter;
@@ -477,8 +477,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
         }
         finally {
-            if (p instanceof GridLoadCacheCloseablePredicate)
-                ((GridLoadCacheCloseablePredicate)p).onClose();
+            if (p instanceof PlatformCacheEntryFilter)
+                ((PlatformCacheEntryFilter)p).onClose();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d0aff43/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryCloseableScanBiPredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryCloseableScanBiPredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryCloseableScanBiPredicate.java
deleted file mode 100644
index d6c56e2..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryCloseableScanBiPredicate.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.query;
-
-import org.apache.ignite.lang.IgniteBiPredicate;
-
-/**
- * Special version of bi-predicate for cache scan queries with close callback.
- */
-public interface CacheQueryCloseableScanBiPredicate<K, V> extends IgniteBiPredicate<K, V> {
-    /**
-     * Callback invoked when predicate is not longer needed.
-     */
-    public void onClose();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d0aff43/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 4206230..b3f8720 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -17,29 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.sql.SQLException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentMap;
-import javax.cache.Cache;
-import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -71,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnrese
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.datastructures.GridCacheSetItemKey;
 import org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
@@ -112,6 +90,30 @@ import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+import javax.cache.Cache;
+import javax.cache.expiry.ExpiryPolicy;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.sql.SQLException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentMap;
+
 import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
@@ -1010,8 +1012,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     heapIt.close();
                 }
                 finally {
-                    if (keyValFilter instanceof CacheQueryCloseableScanBiPredicate)
-                        ((CacheQueryCloseableScanBiPredicate)keyValFilter).onClose();
+                    if (keyValFilter instanceof PlatformCacheEntryFilter)
+                        ((PlatformCacheEntryFilter)keyValFilter).onClose();
                 }
             }
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/6d0aff43/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
index bd3e281..94de2bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
@@ -17,13 +17,14 @@
 
 package org.apache.ignite.internal.processors.platform.cache;
 
-import org.apache.ignite.internal.processors.cache.GridLoadCacheCloseablePredicate;
-import org.apache.ignite.internal.processors.cache.query.CacheQueryCloseableScanBiPredicate;
+import org.apache.ignite.lang.IgniteBiPredicate;
 
 /**
  * Platform cache entry filter interface.
  */
-public interface PlatformCacheEntryFilter<K, V> extends GridLoadCacheCloseablePredicate<K, V>,
-    CacheQueryCloseableScanBiPredicate<K, V> {
-    // No-op.
+public interface PlatformCacheEntryFilter<K, V> extends IgniteBiPredicate<K, V> {
+    /**
+     * Callback invoked when filter is no longer needed.
+     */
+    public void onClose();
 }
\ No newline at end of file


[09/29] ignite git commit: IGNITE-1337: Moved "common" CPP project to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/win/include/ignite/common/common.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/win/include/ignite/common/common.h b/modules/platform/src/main/cpp/common/os/win/include/ignite/common/common.h
new file mode 100644
index 0000000..9e57bde
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/win/include/ignite/common/common.h
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+#ifndef _IGNITE_COMMON_COMMON
+#define _IGNITE_COMMON_COMMON
+
+#define IGNITE_EXPORT __declspec(dllexport)
+#define IGNITE_IMPORT __declspec(dllimport)
+#define IGNITE_CALL __stdcall
+
+#define IGNITE_IMPORT_EXPORT IGNITE_EXPORT
+
+#include <iostream>
+
+#define IGNITE_TRACE_ALLOC(addr) \
+    std::cout << "ALLOC " << __FILE__ << "(" << __LINE__ << "): 0x" << (void*)addr << std::endl;
+
+/**
+ * Common construction to disable copy constructor and assignment for class.
+ */
+#define IGNITE_NO_COPY_ASSIGNMENT(cls) \
+    cls(const cls& src); \
+    cls& operator= (const cls& other); 
+
+namespace ignite
+{
+    namespace common
+    {
+        /**
+         * Helper class to manage attached threads.
+         */
+        class AttachHelper 
+        {
+        public:                       
+            /**
+             * Callback invoked on successful thread attach ot JVM.
+             */
+            static void OnThreadAttach();
+        };   
+    }
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/win/include/ignite/common/concurrent_os.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/win/include/ignite/common/concurrent_os.h b/modules/platform/src/main/cpp/common/os/win/include/ignite/common/concurrent_os.h
new file mode 100644
index 0000000..0a47beb
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/win/include/ignite/common/concurrent_os.h
@@ -0,0 +1,406 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_COMMON_CONCURRENT_OS
+#define _IGNITE_COMMON_CONCURRENT_OS
+
+#include <map>
+#include <stdint.h>
+#include <windows.h>
+
+#include "ignite/common/common.h"
+
+namespace ignite
+{
+    namespace common 
+    {
+        namespace concurrent 
+        {
+            /**
+             * Static class to manage memory visibility semantics. 
+             */
+            class IGNITE_IMPORT_EXPORT Memory {
+            public:
+                /**
+                 * Full fence. 
+                 */
+                static void Fence();
+            };
+
+            /**
+             * Critical section.
+             */
+            class IGNITE_IMPORT_EXPORT CriticalSection {
+            public:
+                /**
+                 * Constructor.
+                 */
+                CriticalSection();
+
+                /**
+                 * Destructor. 
+                 */
+                ~CriticalSection();
+
+                /**
+                 * Enter critical section.
+                 */
+                void Enter();
+
+                /**
+                 * Leave critical section.
+                 */
+                void Leave();
+            private:
+                /** Handle. */
+                CRITICAL_SECTION* hnd;
+
+                IGNITE_NO_COPY_ASSIGNMENT(CriticalSection)
+            };
+
+            /**
+             * Special latch with count = 1.
+             */
+            class IGNITE_IMPORT_EXPORT SingleLatch
+            {                
+            public:
+                /**
+                 * Constructor.
+                 */
+                SingleLatch();
+
+                /**
+                 * Destructor.
+                 */
+                ~SingleLatch();
+
+                /**
+                 * Perform the countdown.
+                 */
+                void CountDown();
+
+                /**
+                 * Await the countdown.
+                 */
+                void Await();
+            private:
+                /** Handle. */
+                void* hnd;
+
+                IGNITE_NO_COPY_ASSIGNMENT(SingleLatch)
+            };
+
+            /**
+             * Primitives for atomic access.
+             */
+            class IGNITE_IMPORT_EXPORT Atomics
+            {
+            public:
+                /**
+                 * Update the 32-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return True if update occurred as a result of this call, false otherwise.
+                 */
+                static bool CompareAndSet32(int32_t* ptr, int32_t expVal, int32_t newVal);
+
+                /**
+                 * Update the 32-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return Value which were observed during CAS attempt.
+                 */
+                static int32_t CompareAndSet32Val(int32_t* ptr, int32_t expVal, int32_t newVal);
+                
+                /**
+                 * Increment 32-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after increment.
+                 */
+                static int32_t IncrementAndGet32(int32_t* ptr);
+
+                /**
+                 * Decrement 32-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after decrement.
+                 */
+                static int32_t DecrementAndGet32(int32_t* ptr);
+
+                /**
+                 * Update the 64-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return True if update occurred as a result of this call, false otherwise.
+                 */
+                static bool CompareAndSet64(int64_t* ptr, int64_t expVal, int64_t newVal);
+
+                /**
+                 * Update the 64-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return Value which were observed during CAS attempt.
+                 */
+                static int64_t CompareAndSet64Val(int64_t* ptr, int64_t expVal, int64_t newVal);
+                
+                /**
+                 * Increment 64-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after increment.
+                 */
+                static int64_t IncrementAndGet64(int64_t* ptr);
+
+                /**
+                 * Decrement 64-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after decrement.
+                 */
+                static int64_t DecrementAndGet64(int64_t* ptr);
+            };
+
+            /**
+             * Thread-local entry.
+             */
+            class IGNITE_IMPORT_EXPORT ThreadLocalEntry
+            {
+            public:
+                /**
+                 * Virtual destructor to allow for correct typed entries cleanup.
+                 */
+                virtual ~ThreadLocalEntry()
+                {
+                    // No-op.
+                }
+            };
+
+            /**
+             * Typed thread-local entry.
+             */
+            template<typename T>
+            class IGNITE_IMPORT_EXPORT ThreadLocalTypedEntry : public ThreadLocalEntry
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param val Value.
+                 */
+                ThreadLocalTypedEntry(T val) : val(val)
+                {
+                    // No-op.
+                }
+                
+                ~ThreadLocalTypedEntry()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get value.
+                 *
+                 * @return Value.
+                 */
+                T Get()
+                {
+                    return val;
+                }
+            private:
+                /** Value. */
+                T val; 
+            };
+
+            /**
+             * Thread-local abstraction.
+             */
+            class IGNITE_IMPORT_EXPORT ThreadLocal
+            {
+            public:
+                /**
+                 * Allocate thread-local index. Invoked once on DLL process attach.
+                 *
+                 * @return True if allocation was successful.
+                 */
+                static bool OnProcessAttach();
+
+                /**
+                 * Release thread-local entry. Invoked on DLL thread detach.
+                 */
+                static void OnThreadDetach();
+
+                /**
+                 * Release thread-local index. Invoked once on DLL process detach.
+                 */
+                static void OnProcessDetach();
+
+                /**
+                 * Get next available index to be used in thread-local storage.
+                 *
+                 * @return Index.
+                 */
+                static int32_t NextIndex();
+
+                /**
+                 * Get value by index.
+                 *
+                 * @param idx Index.
+                 * @return Value associated with the index or NULL.
+                 */
+                template<typename T>
+                static T Get(int32_t idx)
+                {
+                    void* winVal = Get0();
+
+                    if (winVal)
+                    {
+                        std::map<int32_t, ThreadLocalEntry*>* map = 
+                            static_cast<std::map<int32_t, ThreadLocalEntry*>*>(winVal);
+
+                        ThreadLocalTypedEntry<T>* entry = static_cast<ThreadLocalTypedEntry<T>*>((*map)[idx]);
+                        
+                        if (entry)
+                            return entry->Get();
+                    }
+
+                    return T();
+                }
+
+                /**
+                 * Set value at the given index.
+                 *
+                 * @param idx Index.
+                 * @param val Value to be associated with the index.
+                 */
+                template<typename T>
+                static void Set(int32_t idx, const T& val)
+                {
+                    void* winVal = Get0();
+
+                    if (winVal)
+                    {
+                        std::map<int32_t, ThreadLocalEntry*>* map = 
+                            static_cast<std::map<int32_t, ThreadLocalEntry*>*>(winVal);
+
+                        ThreadLocalEntry* appVal = (*map)[idx];
+
+                        if (appVal)
+                            delete appVal;
+
+                        (*map)[idx] = new ThreadLocalTypedEntry<T>(val);
+                    }
+                    else
+                    {
+                        std::map<int32_t, ThreadLocalEntry*>* map = new std::map<int32_t, ThreadLocalEntry*>();
+
+                        Set0(map);
+
+                        (*map)[idx] = new ThreadLocalTypedEntry<T>(val);
+                    }
+                }
+
+                /**
+                 * Remove value at the given index.
+                 *
+                 * @param idx Index.
+                 */
+                static void Remove(int32_t idx);
+
+            private:
+                /**
+                 * Internal get routine.
+                 *
+                 * @param Associated value.
+                 */
+                static void* Get0();
+
+                /**
+                 * Internal set routine.
+                 *
+                 * @param ptr Pointer.
+                 */
+                static void Set0(void* ptr);
+
+                /**
+                 * Internal thread-local map clear routine.
+                 *
+                 * @param mapPtr Pointer to map.
+                 */
+                static void Clear0(void* mapPtr);
+            };
+
+            /**
+             * Thread-local instance. Simplifies API avoiding direct index allocations.
+             */
+            template<typename T>
+            class IGNITE_IMPORT_EXPORT ThreadLocalInstance
+            {
+            public:
+                /**
+                 * Constructor.
+                 */
+                ThreadLocalInstance() : idx(ThreadLocal::NextIndex())
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get value.
+                 *
+                 * @return Value.
+                 */
+                T Get()
+                {
+                    return ThreadLocal::Get<T>(idx);
+                }
+
+                /**
+                 * Set instance.
+                 *
+                 * @param val Value.
+                 */
+                void Set(const T& val)
+                {
+                    ThreadLocal::Set<T>(idx, val);
+                }
+
+                /**
+                 * Remove instance.
+                 */
+                void Remove()
+                {
+                    ThreadLocal::Remove(idx);
+                }
+
+            private:
+                /** Index. */
+                int32_t idx; 
+            };
+        }
+    }
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/win/src/common.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/win/src/common.cpp b/modules/platform/src/main/cpp/common/os/win/src/common.cpp
new file mode 100644
index 0000000..e83e736
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/win/src/common.cpp
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+#include <windows.h>
+
+#include "ignite/common/common.h"
+#include "ignite/common/concurrent.h"
+#include "ignite/common/java.h"
+
+using namespace ignite::common::concurrent;
+using namespace ignite::common::java;
+
+namespace ignite
+{
+    namespace common
+    {
+        void AttachHelper::OnThreadAttach()
+        {
+            // No-op.
+        }
+    }
+}
+
+BOOL WINAPI DllMain(_In_ HINSTANCE hinstDLL, _In_ DWORD fdwReason, _In_ LPVOID lpvReserved)
+{
+    switch (fdwReason)
+    {
+        case DLL_PROCESS_ATTACH:
+            if (!ThreadLocal::OnProcessAttach())
+                return FALSE;
+
+            break;
+
+        case DLL_THREAD_DETACH:
+            ThreadLocal::OnThreadDetach();
+
+            JniContext::Detach();
+
+            break;
+
+        case DLL_PROCESS_DETACH:
+            ThreadLocal::OnProcessDetach();
+
+            break;
+
+        default:
+            break;
+    }
+
+    return TRUE;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/win/src/concurrent_os.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/win/src/concurrent_os.cpp b/modules/platform/src/main/cpp/common/os/win/src/concurrent_os.cpp
new file mode 100644
index 0000000..a21f7ec
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/win/src/concurrent_os.cpp
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+
+#include "ignite/common/concurrent_os.h"
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace concurrent
+        {
+            /** Thread-local index for Windows. */
+            DWORD winTlsIdx;
+
+            void Memory::Fence() {
+                MemoryBarrier();
+            }
+
+            CriticalSection::CriticalSection() : hnd(new CRITICAL_SECTION) {
+                InitializeCriticalSection(hnd);
+
+                Memory::Fence();
+            }
+
+            CriticalSection::~CriticalSection() {
+                Memory::Fence();
+
+                delete hnd;
+            }
+
+            void CriticalSection::Enter() {
+                Memory::Fence();
+
+                EnterCriticalSection(hnd);
+            }
+
+            void CriticalSection::Leave() {
+                Memory::Fence();
+
+                LeaveCriticalSection(hnd);
+            }
+
+            SingleLatch::SingleLatch() : hnd(CreateEvent(NULL, TRUE, FALSE, NULL))
+            {
+                Memory::Fence();
+            }
+
+            SingleLatch::~SingleLatch()
+            {
+                Memory::Fence();
+
+                CloseHandle(hnd);
+            }
+
+            void SingleLatch::CountDown()
+            {
+                SetEvent(hnd);
+            }
+
+            void SingleLatch::Await()
+            {
+                WaitForSingleObject(hnd, INFINITE);
+            }
+
+            bool Atomics::CompareAndSet32(int32_t* ptr, int32_t expVal, int32_t newVal)
+            {
+                return CompareAndSet32Val(ptr, expVal, newVal) == expVal;
+            }
+
+            int32_t Atomics::CompareAndSet32Val(int32_t* ptr, int32_t expVal, int32_t newVal)
+            {
+                return InterlockedCompareExchange(reinterpret_cast<LONG*>(ptr), newVal, expVal);
+            }
+
+            int32_t Atomics::IncrementAndGet32(int32_t* ptr)
+            {
+                return InterlockedIncrement(reinterpret_cast<LONG*>(ptr));
+            }
+
+            int32_t Atomics::DecrementAndGet32(int32_t* ptr)
+            {
+                return InterlockedDecrement(reinterpret_cast<LONG*>(ptr));
+            }
+
+            bool Atomics::CompareAndSet64(int64_t* ptr, int64_t expVal, int64_t newVal)
+            {
+                return CompareAndSet64Val(ptr, expVal, newVal) == expVal;
+            }
+
+            int64_t Atomics::CompareAndSet64Val(int64_t* ptr, int64_t expVal, int64_t newVal)
+            {
+                return InterlockedCompareExchange64(reinterpret_cast<LONG64*>(ptr), newVal, expVal);
+            }
+
+            int64_t Atomics::IncrementAndGet64(int64_t* ptr)
+            {
+                return InterlockedIncrement64(reinterpret_cast<LONG64*>(ptr));
+            }
+
+            int64_t Atomics::DecrementAndGet64(int64_t* ptr)
+            {
+                return InterlockedDecrement64(reinterpret_cast<LONG64*>(ptr));
+            }
+            
+            bool ThreadLocal::OnProcessAttach()
+            {
+                return (winTlsIdx = TlsAlloc()) != TLS_OUT_OF_INDEXES;
+            }
+
+            void ThreadLocal::OnThreadDetach()
+            {
+                if (winTlsIdx != TLS_OUT_OF_INDEXES)
+                {
+                    void* mapPtr = Get0();
+
+                    Clear0(mapPtr);
+                }
+            }
+
+            void ThreadLocal::OnProcessDetach()
+            {
+                if (winTlsIdx != TLS_OUT_OF_INDEXES)
+                    TlsFree(winTlsIdx);
+            }
+
+            void* ThreadLocal::Get0()
+            {
+                return TlsGetValue(winTlsIdx);
+            }
+
+            void ThreadLocal::Set0(void* ptr)
+            {
+                TlsSetValue(winTlsIdx, ptr);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/project/README.TXT
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/README.TXT b/modules/platform/src/main/cpp/common/project/README.TXT
new file mode 100644
index 0000000..97f4c64
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/project/README.TXT
@@ -0,0 +1 @@
+Contains IDE projects artifacts.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/project/vs/README.TXT
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/README.TXT b/modules/platform/src/main/cpp/common/project/vs/README.TXT
new file mode 100644
index 0000000..f4fb456
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/project/vs/README.TXT
@@ -0,0 +1 @@
+Contains Visual Studio project artifacts.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/common.vcxproj b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
new file mode 100644
index 0000000..478932f
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj
@@ -0,0 +1,202 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Debug|Win32">
+      <Configuration>Debug</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Debug|x64">
+      <Configuration>Debug</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|x64">
+      <Configuration>Release</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+  </ItemGroup>
+  <PropertyGroup Label="Globals">
+    <ProjectGuid>{4F7E4917-4612-4B96-9838-025711ADE391}</ProjectGuid>
+    <Keyword>Win32Proj</Keyword>
+    <RootNamespace>common</RootNamespace>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="Configuration">
+    <ConfigurationType>DynamicLibrary</ConfigurationType>
+    <UseDebugLibraries>true</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
+    <ConfigurationType>DynamicLibrary</ConfigurationType>
+    <UseDebugLibraries>true</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
+    <ConfigurationType>DynamicLibrary</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>DynamicLibrary</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
+  <ImportGroup Label="ExtensionSettings">
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <PropertyGroup Label="UserMacros" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
+    <LinkIncremental>true</LinkIncremental>
+    <TargetName>ignite.common</TargetName>
+    <OutDir>$(SolutionDir)$(Platform)\$(Configuration)\</OutDir>
+    <IntDir>$(Platform)\$(Configuration)\</IntDir>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <TargetName>ignite.common</TargetName>
+    <OutDir>$(SolutionDir)$(Platform)\$(Configuration)\</OutDir>
+    <IntDir>$(Platform)\$(Configuration)\</IntDir>
+    <LinkIncremental>true</LinkIncremental>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <LinkIncremental>false</LinkIncremental>
+    <TargetName>ignite.common</TargetName>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <TargetName>ignite.common</TargetName>
+    <OutDir>$(SolutionDir)$(Platform)\$(Configuration)\</OutDir>
+    <IntDir>$(Platform)\$(Configuration)\</IntDir>
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
+    <ClCompile>
+      <PrecompiledHeader>NotUsing</PrecompiledHeader>
+      <WarningLevel>Level3</WarningLevel>
+      <Optimization>Disabled</Optimization>
+      <PreprocessorDefinitions>WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Windows</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+      <DelayLoadDLLs>jvm.dll</DelayLoadDLLs>
+      <ModuleDefinitionFile>module.def</ModuleDefinitionFile>
+      <OptimizeReferences>false</OptimizeReferences>
+      <EnableCOMDATFolding>false</EnableCOMDATFolding>
+      <LinkTimeCodeGeneration>Default</LinkTimeCodeGeneration>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <SDLCheck>false</SDLCheck>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <PreprocessorDefinitions>WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
+    </ClCompile>
+    <Link>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <AdditionalLibraryDirectories>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <OptimizeReferences>true</OptimizeReferences>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <DelayLoadDLLs>jvm.dll</DelayLoadDLLs>
+      <LinkTimeCodeGeneration>Default</LinkTimeCodeGeneration>
+      <ModuleDefinitionFile>module.def</ModuleDefinitionFile>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>NotUsing</PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <InlineFunctionExpansion>AnySuitable</InlineFunctionExpansion>
+      <FavorSizeOrSpeed>Speed</FavorSizeOrSpeed>
+      <OmitFramePointers>true</OmitFramePointers>
+      <BufferSecurityCheck>false</BufferSecurityCheck>
+      <StringPooling>true</StringPooling>
+    </ClCompile>
+    <Link>
+      <SubSystem>Windows</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+      <DelayLoadDLLs>jvm.dll</DelayLoadDLLs>
+      <ModuleDefinitionFile>module.def</ModuleDefinitionFile>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <Optimization>Full</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <SDLCheck>false</SDLCheck>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <InlineFunctionExpansion>AnySuitable</InlineFunctionExpansion>
+      <FavorSizeOrSpeed>Speed</FavorSizeOrSpeed>
+      <OmitFramePointers>true</OmitFramePointers>
+      <StringPooling>true</StringPooling>
+      <BufferSecurityCheck>false</BufferSecurityCheck>
+      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+    </ClCompile>
+    <Link>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalLibraryDirectories>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <DelayLoadDLLs>jvm.dll</DelayLoadDLLs>
+      <ModuleDefinitionFile>module.def</ModuleDefinitionFile>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemGroup>
+    <ClInclude Include="..\..\include\ignite\common\concurrent.h" />
+    <ClInclude Include="..\..\include\ignite\common\exports.h" />
+    <ClInclude Include="..\..\include\ignite\common\java.h" />
+    <ClInclude Include="..\..\os\win\include\ignite\common\common.h" />
+    <ClInclude Include="..\..\os\win\include\ignite\common\concurrent_os.h" />
+    <ClInclude Include="targetver.h" />
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\os\win\src\common.cpp" />
+    <ClCompile Include="..\..\os\win\src\concurrent_os.cpp" />
+    <ClCompile Include="..\..\src\concurrent.cpp" />
+    <ClCompile Include="..\..\src\exports.cpp" />
+    <ClCompile Include="..\..\src\java.cpp" />
+  </ItemGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
+  <ImportGroup Label="ExtensionTargets">
+  </ImportGroup>
+  <ItemGroup>
+    <None Include="module.def" />
+  </ItemGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/project/vs/common.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/common.vcxproj.filters b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj.filters
new file mode 100644
index 0000000..3d4ae54
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/project/vs/common.vcxproj.filters
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup>
+    <Filter Include="Misc">
+      <UniqueIdentifier>{1dbec2be-5cb4-4f70-aef6-b4627d39b99b}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Code">
+      <UniqueIdentifier>{93995380-89BD-4b04-88EB-625FBE52EBFB}</UniqueIdentifier>
+      <Extensions>h;hh;hpp;hxx;hm;inl;inc;xsd</Extensions>
+    </Filter>
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\..\os\win\include\ignite\common\common.h">
+      <Filter>Code</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\os\win\include\ignite\common\concurrent_os.h">
+      <Filter>Code</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\common\exports.h">
+      <Filter>Code</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\common\java.h">
+      <Filter>Code</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\common\concurrent.h">
+      <Filter>Code</Filter>
+    </ClInclude>
+    <ClInclude Include="targetver.h">
+      <Filter>Misc</Filter>
+    </ClInclude>
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\os\win\src\common.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\concurrent.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\os\win\src\concurrent_os.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\exports.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\java.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="module.def">
+      <Filter>Misc</Filter>
+    </None>
+  </ItemGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/module.def b/modules/platform/src/main/cpp/common/project/vs/module.def
new file mode 100644
index 0000000..d9e8d2b
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/project/vs/module.def
@@ -0,0 +1,99 @@
+LIBRARY ignite.common.dll
+EXPORTS
+IgniteReallocate @1 
+IgniteIgnitionStart @2 
+IgniteIgnitionInstance @3 
+IgniteIgnitionEnvironmentPointer @4 
+IgniteIgnitionStop @5 
+IgniteIgnitionStopAll @6 
+IgniteTargetOutLong @7
+IgniteProcessorReleaseStart @8 
+IgniteProcessorProjection @9 
+IgniteProcessorCache @10 
+IgniteProcessorCreateCache @11 
+IgniteProcessorGetOrCreateCache @12 
+IgniteProcessorAffinity @13 
+IgniteProcessorDataStreamer @14 
+IgniteProcessorTransactions @15 
+IgniteProcessorServices @16
+IgniteTargetInStreamOutObject @17 
+IgniteTargetInStreamOutLong @18 
+IgniteTargetOutStream @19 
+IgniteTargetInStreamOutStream @20 
+IgniteTargetInObjectStreamOutStream @21 
+IgniteTargetListenFuture @22 
+IgniteTargetListenFutureForOperation @23 
+IgniteAffinityPartitions @24 
+IgniteCacheWithSkipStore @25 
+IgniteCacheWithNoRetries @26 
+IgniteCacheWithExpiryPolicy @27 
+IgniteCacheWithAsync @28 
+IgniteCacheWithKeepPortable @29 
+IgniteCacheClear @30 
+IgniteCacheRemoveAll @31 
+IgniteCacheOutOpQueryCursor @32 
+IgniteCacheOutOpContinuousQuery @33 
+IgniteCacheIterator @34 
+IgniteCacheLocalIterator @35 
+IgniteCacheEnterLock @36 
+IgniteCacheExitLock @37 
+IgniteCacheTryEnterLock @38 
+IgniteCacheCloseLock @39 
+IgniteCacheRebalance @40 
+IgniteCacheSize @41 
+IgniteCacheStoreCallbackInvoke @42 
+IgniteComputeWithNoFailover @43 
+IgniteComputeWithTimeout @44 
+IgniteComputeExecuteNative @45 
+IgniteContinuousQueryClose @46 
+IgniteContinuousQueryGetInitialQueryCursor @47 
+IgniteDataStreamerListenTopology @48 
+IgniteDataStreamerAllowOverwriteGet @49 
+IgniteDataStreamerAllowOverwriteSet @50 
+IgniteDataStreamerSkipStoreGet @51 
+IgniteDataStreamerSkipStoreSet @52 
+IgniteDataStreamerPerNodeBufferSizeGet @53 
+IgniteDataStreamerPerNodeBufferSizeSet @54 
+IgniteDataStreamerPerNodeParallelOperationsGet @55 
+IgniteDataStreamerPerNodeParallelOperationsSet @56 
+IgniteMessagingWithAsync @57 
+IgniteProjectionForOthers @58 
+IgniteProjectionForRemotes @59 
+IgniteProjectionForDaemons @60 
+IgniteProjectionForRandom @61 
+IgniteProjectionForOldest @62 
+IgniteProjectionForYoungest @63 
+IgniteProcessorCompute @64 
+IgniteProcessorMessage @65 
+IgniteProcessorEvents @66 
+IgniteProjectionResetMetrics @67 
+IgniteProjectionOutOpRet @68 
+IgniteQueryCursorIterator @69 
+IgniteQueryCursorClose @70 
+IgniteTransactionsStart @71 
+IgniteTransactionsCommit @72 
+IgniteTransactionsCommitAsync @73 
+IgniteTransactionsRollback @74 
+IgniteTransactionsRollbackAsync @75 
+IgniteTransactionsClose @76 
+IgniteTransactionsState @77 
+IgniteTransactionsSetRollbackOnly @78 
+IgniteTransactionsResetMetrics @79 
+IgniteAcquire @80 
+IgniteRelease @81 
+IgniteThrowToJava @82 
+IgniteHandlersSize @83 
+IgniteCreateContext @84 
+IgniteDeleteContext @85 
+IgniteDestroyJvm @86 
+IgniteEventsWithAsync @87 
+IgniteEventsStopLocalListen @88 
+IgniteEventsLocalListen @89 
+IgniteEventsIsEnabled @90 
+IgniteTargetOutObject @91 
+IgniteServicesWithAsync @92
+IgniteServicesWithServerKeepPortable @93
+IgniteServicesCancel @94
+IgniteServicesCancelAll @95
+IgniteServicesGetServiceProxy @96
+IgniteProcessorExtensions @97
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/project/vs/targetver.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/project/vs/targetver.h b/modules/platform/src/main/cpp/common/project/vs/targetver.h
new file mode 100644
index 0000000..4bea158
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/project/vs/targetver.h
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+// Including SDKDDKVer.h defines the highest available Windows platform.
+
+// If you wish to build your application for a previous Windows platform, include WinSDKVer.h and
+// set the _WIN32_WINNT macro to the platform you wish to support before including SDKDDKVer.h.
+
+#include <SDKDDKVer.h>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/src/concurrent.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/src/concurrent.cpp b/modules/platform/src/main/cpp/common/src/concurrent.cpp
new file mode 100644
index 0000000..3f85b65
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/src/concurrent.cpp
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+#include "ignite/common/concurrent.h"
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace concurrent
+        {
+            /** Thread-local index generator for application. */
+            int32_t appTlsIdxGen = 0;
+
+            int32_t ThreadLocal::NextIndex()
+            {
+                return Atomics::IncrementAndGet32(&appTlsIdxGen);
+            }
+
+            void ThreadLocal::Remove(int32_t idx)
+            {
+                void* val = Get0();
+
+                if (val)
+                {
+                    std::map<int32_t, ThreadLocalEntry*>* map =
+                        static_cast<std::map<int32_t, ThreadLocalEntry*>*>(val);
+
+                    ThreadLocalEntry* appVal = (*map)[idx];
+
+                    if (appVal)
+                        delete appVal;
+
+                    map->erase(idx);
+
+                    if (map->size() == 0)
+                    {
+                        delete map;
+
+                        Set0(NULL);
+                    }
+                }
+            }
+
+            void ThreadLocal::Clear0(void* mapPtr)
+            {
+                if (mapPtr)
+                {
+                    std::map<int32_t, ThreadLocalEntry*>* map =
+                        static_cast<std::map<int32_t, ThreadLocalEntry*>*>(mapPtr);
+
+                    for (std::map<int32_t, ThreadLocalEntry*>::iterator it = map->begin(); it != map->end(); ++it)
+                        delete it->second;
+
+                    delete map;
+                }
+            }
+
+            SharedPointerImpl::SharedPointerImpl(void* ptr) : ptr(ptr), refCnt(1)
+            {
+                Memory::Fence();
+            }
+
+            void* SharedPointerImpl::Pointer()
+            {
+                return ptr;
+            }
+
+            void SharedPointerImpl::Increment()
+            {
+                Atomics::IncrementAndGet32(&refCnt);
+            }
+
+            bool SharedPointerImpl::Decrement()
+            {
+                return Atomics::DecrementAndGet32(&refCnt) == 0;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/src/exports.cpp b/modules/platform/src/main/cpp/common/src/exports.cpp
new file mode 100644
index 0000000..10e4801
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/src/exports.cpp
@@ -0,0 +1,413 @@
+/*
+ * 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.
+ */
+
+#include "ignite/common/exports.h"
+#include "ignite/common/java.h"
+
+namespace gcj = ignite::common::java;
+
+/* --- Target methods. --- */
+extern "C" {
+    int IGNITE_CALL IgniteReallocate(long long memPtr, int cap) {
+        return gcj::JniContext::Reallocate(memPtr, cap);
+    }
+
+    void* IGNITE_CALL IgniteIgnitionStart(gcj::JniContext* ctx, char* cfgPath, char* gridName, int factoryId, long long dataPtr) {
+        return ctx->IgnitionStart(cfgPath, gridName, factoryId, dataPtr);
+    }
+
+	void* IGNITE_CALL IgniteIgnitionInstance(gcj::JniContext* ctx, char* gridName) {
+        return ctx->IgnitionInstance(gridName);
+    }
+
+    long long IGNITE_CALL IgniteIgnitionEnvironmentPointer(gcj::JniContext* ctx, char* gridName) {
+        return ctx->IgnitionEnvironmentPointer(gridName);
+    }
+
+	bool IGNITE_CALL IgniteIgnitionStop(gcj::JniContext* ctx, char* gridName, bool cancel) {
+        return ctx->IgnitionStop(gridName, cancel);
+    }
+
+	void IGNITE_CALL IgniteIgnitionStopAll(gcj::JniContext* ctx, bool cancel) {
+        return ctx->IgnitionStopAll(cancel);
+    }
+
+    void IGNITE_CALL IgniteProcessorReleaseStart(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProcessorReleaseStart(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProcessorProjection(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProcessorProjection(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProcessorCache(gcj::JniContext* ctx, void* obj, char* name) {
+        return ctx->ProcessorCache(static_cast<jobject>(obj), name);
+    }
+
+    void* IGNITE_CALL IgniteProcessorCreateCache(gcj::JniContext* ctx, void* obj, char* name) {
+        return ctx->ProcessorCreateCache(static_cast<jobject>(obj), name);
+    }
+
+    void* IGNITE_CALL IgniteProcessorGetOrCreateCache(gcj::JniContext* ctx, void* obj, char* name) {
+        return ctx->ProcessorGetOrCreateCache(static_cast<jobject>(obj), name);
+    }
+
+    void* IGNITE_CALL IgniteProcessorAffinity(gcj::JniContext* ctx, void* obj, char* name) {
+        return ctx->ProcessorAffinity(static_cast<jobject>(obj), name);
+    }
+
+    void*IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable) {
+        return ctx->ProcessorDataStreamer(static_cast<jobject>(obj), name, keepPortable);
+    }
+    
+    void* IGNITE_CALL IgniteProcessorTransactions(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProcessorTransactions(static_cast<jobject>(obj));
+    }
+        
+    void* IGNITE_CALL IgniteProcessorCompute(gcj::JniContext* ctx, void* obj, void* prj) {
+        return ctx->ProcessorCompute(static_cast<jobject>(obj), static_cast<jobject>(prj));
+    }
+
+    void* IGNITE_CALL IgniteProcessorMessage(gcj::JniContext* ctx, void* obj, void* prj) {
+        return ctx->ProcessorMessage(static_cast<jobject>(obj), static_cast<jobject>(prj));
+    }
+
+    void* IGNITE_CALL IgniteProcessorEvents(gcj::JniContext* ctx, void* obj, void* prj) {
+        return ctx->ProcessorEvents(static_cast<jobject>(obj), static_cast<jobject>(prj));
+    }
+
+    void* IGNITE_CALL IgniteProcessorServices(gcj::JniContext* ctx, void* obj, void* prj) {
+        return ctx->ProcessorServices(static_cast<jobject>(obj), static_cast<jobject>(prj));
+    }
+
+    void* IGNITE_CALL IgniteProcessorExtensions(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProcessorExtensions(static_cast<jobject>(obj));
+    }
+
+    long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
+        return ctx->TargetInStreamOutLong(static_cast<jobject>(obj), opType, memPtr);
+    }
+
+    void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr) {
+        ctx->TargetInStreamOutStream(static_cast<jobject>(obj), opType, inMemPtr, outMemPtr);
+    }
+
+    void* IGNITE_CALL IgniteTargetInStreamOutObject(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
+        return ctx->TargetInStreamOutObject(static_cast<jobject>(obj), opType, memPtr);
+    }
+
+    void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr) {
+        ctx->TargetInObjectStreamOutStream(static_cast<jobject>(obj), opType, arg, inMemPtr, outMemPtr);
+    }
+    
+    long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType) {
+        return ctx->TargetOutLong(static_cast<jobject>(obj), opType);
+    }
+
+    void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
+        ctx->TargetOutStream(static_cast<jobject>(obj), opType, memPtr);
+    }
+
+    void* IGNITE_CALL IgniteTargetOutObject(gcj::JniContext* ctx, void* obj, int opType) {
+        return ctx->TargetOutObject(static_cast<jobject>(obj), opType);
+    }
+
+    void IGNITE_CALL IgniteTargetListenFuture(gcj::JniContext* ctx, void* obj, long long futId, int typ) {
+        ctx->TargetListenFuture(static_cast<jobject>(obj), futId, typ);
+    }
+
+    void IGNITE_CALL IgniteTargetListenFutureForOperation(gcj::JniContext* ctx, void* obj, long long futId, int typ, int opId) {
+        ctx->TargetListenFutureForOperation(static_cast<jobject>(obj), futId, typ, opId);
+    }
+
+    int IGNITE_CALL IgniteAffinityPartitions(gcj::JniContext* ctx, void* obj) {
+        return ctx->AffinityPartitions(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteCacheWithSkipStore(gcj::JniContext* ctx, void* obj) {
+        return ctx->CacheWithSkipStore(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteCacheWithNoRetries(gcj::JniContext* ctx, void* obj) {
+        return ctx->CacheWithNoRetries(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteCacheWithExpiryPolicy(gcj::JniContext* ctx, void* obj, long long create, long long update, long long access) {
+        return ctx->CacheWithExpiryPolicy(static_cast<jobject>(obj), create, update, access);
+    }
+
+    void* IGNITE_CALL IgniteCacheWithAsync(gcj::JniContext* ctx, void* obj) {
+        return ctx->CacheWithAsync(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteCacheWithKeepPortable(gcj::JniContext* ctx, void* obj)
+    {
+        return ctx->CacheWithKeepPortable(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteCacheClear(gcj::JniContext* ctx, void* obj) {
+        ctx->CacheClear(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteCacheRemoveAll(gcj::JniContext* ctx, void* obj) {
+        ctx->CacheRemoveAll(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteCacheOutOpQueryCursor(gcj::JniContext* ctx, void* obj, int type, long long memPtr) {
+        return ctx->CacheOutOpQueryCursor(static_cast<jobject>(obj), type, memPtr);
+    }
+
+    void* IGNITE_CALL IgniteCacheOutOpContinuousQuery(gcj::JniContext* ctx, void* obj, int type, long long memPtr) {
+        return ctx->CacheOutOpContinuousQuery(static_cast<jobject>(obj), type, memPtr);
+    }
+
+    void* IGNITE_CALL IgniteCacheIterator(gcj::JniContext* ctx, void* obj) {
+        return ctx->CacheIterator(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteCacheLocalIterator(gcj::JniContext* ctx, void* obj, int peekModes) {
+        return ctx->CacheLocalIterator(static_cast<jobject>(obj), peekModes);
+    }
+
+    void IGNITE_CALL IgniteCacheEnterLock(gcj::JniContext* ctx, void* obj, long long id) {
+        ctx->CacheEnterLock(static_cast<jobject>(obj), id);
+    }
+
+    void IGNITE_CALL IgniteCacheExitLock(gcj::JniContext* ctx, void* obj, long long id) {
+        ctx->CacheExitLock(static_cast<jobject>(obj), id);
+    }
+
+    bool IGNITE_CALL IgniteCacheTryEnterLock(gcj::JniContext* ctx, void* obj, long long id, long long timeout) {
+        return ctx->CacheTryEnterLock(static_cast<jobject>(obj), id, timeout);
+    }
+
+    void IGNITE_CALL IgniteCacheCloseLock(gcj::JniContext* ctx, void* obj, long long id) {
+        ctx->CacheCloseLock(static_cast<jobject>(obj), id);
+    }
+
+    void IGNITE_CALL IgniteCacheRebalance(gcj::JniContext* ctx, void* obj, long long futId) {
+        ctx->CacheRebalance(static_cast<jobject>(obj), futId);
+    }
+
+    int IGNITE_CALL IgniteCacheSize(gcj::JniContext* ctx, void* obj, int peekModes, bool loc) {
+        return ctx->CacheSize(static_cast<jobject>(obj), peekModes, loc);
+    }
+
+    void IGNITE_CALL IgniteComputeWithNoFailover(gcj::JniContext* ctx, void* obj) {
+        ctx->ComputeWithNoFailover(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteComputeWithTimeout(gcj::JniContext* ctx, void* obj, long long timeout) {
+        ctx->ComputeWithTimeout(static_cast<jobject>(obj), timeout);
+    }
+
+    void IGNITE_CALL IgniteComputeExecuteNative(gcj::JniContext* ctx, void* obj, long long taskPtr, long long topVer) {
+        ctx->ComputeExecuteNative(static_cast<jobject>(obj), taskPtr, topVer);
+    }
+
+    void IGNITE_CALL IgniteContinuousQueryClose(gcj::JniContext* ctx, void* obj) {
+        ctx->ContinuousQueryClose(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteContinuousQueryGetInitialQueryCursor(gcj::JniContext* ctx, void* obj) {
+        return ctx->ContinuousQueryGetInitialQueryCursor(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteCacheStoreCallbackInvoke(gcj::JniContext* ctx, void* obj, long long memPtr) {
+        ctx->CacheStoreCallbackInvoke(static_cast<jobject>(obj), memPtr);
+    }
+
+    void IGNITE_CALL IgniteDataStreamerListenTopology(gcj::JniContext* ctx, void* obj, long long ptr) {
+        ctx->DataStreamerListenTopology(static_cast<jobject>(obj), ptr);
+    }
+
+    bool IGNITE_CALL IgniteDataStreamerAllowOverwriteGet(gcj::JniContext* ctx, void* obj) {
+        return ctx->DataStreamerAllowOverwriteGet(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteDataStreamerAllowOverwriteSet(gcj::JniContext* ctx, void* obj, bool val) {
+        ctx->DataStreamerAllowOverwriteSet(static_cast<jobject>(obj), val);
+    }
+
+    bool IGNITE_CALL IgniteDataStreamerSkipStoreGet(gcj::JniContext* ctx, void* obj) {
+        return ctx->DataStreamerSkipStoreGet(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteDataStreamerSkipStoreSet(gcj::JniContext* ctx, void* obj, bool val) {
+        ctx->DataStreamerSkipStoreSet(static_cast<jobject>(obj), val);
+    }
+
+    int IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeGet(gcj::JniContext* ctx, void* obj) {
+        return ctx->DataStreamerPerNodeBufferSizeGet(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeSet(gcj::JniContext* ctx, void* obj, int val) {
+        ctx->DataStreamerPerNodeBufferSizeSet(static_cast<jobject>(obj), val);
+    }
+
+    int IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsGet(gcj::JniContext* ctx, void* obj) {
+        return ctx->DataStreamerPerNodeParallelOperationsGet(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsSet(gcj::JniContext* ctx, void* obj, int val) {
+        ctx->DataStreamerPerNodeParallelOperationsSet(static_cast<jobject>(obj), val);
+    }
+
+    void* IGNITE_CALL IgniteMessagingWithAsync(gcj::JniContext* ctx, void* obj) {
+        return ctx->MessagingWithAsync(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionForOthers(gcj::JniContext* ctx, void* obj, void* prj) {
+        return ctx->ProjectionForOthers(static_cast<jobject>(obj), static_cast<jobject>(prj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionForRemotes(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProjectionForRemotes(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionForDaemons(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProjectionForDaemons(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionForRandom(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProjectionForRandom(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionForOldest(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProjectionForOldest(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionForYoungest(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProjectionForYoungest(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteProjectionResetMetrics(gcj::JniContext* ctx, void* obj) {
+        ctx->ProjectionResetMetrics(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteProjectionOutOpRet(gcj::JniContext* ctx, void* obj, int type, long long memPtr) {
+        return ctx->ProjectionOutOpRet(static_cast<jobject>(obj), type, memPtr);
+    }
+
+    void IGNITE_CALL IgniteQueryCursorIterator(gcj::JniContext* ctx, void* obj) {
+        ctx->QueryCursorIterator(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteQueryCursorClose(gcj::JniContext* ctx, void* obj) {
+        ctx->QueryCursorClose(static_cast<jobject>(obj));
+    }
+
+    long long IGNITE_CALL IgniteTransactionsStart(gcj::JniContext* ctx, void* obj, int concurrency, int isolation, long long timeout, int txSize) {
+        return ctx->TransactionsStart(static_cast<jobject>(obj), concurrency, isolation, timeout, txSize);
+    }   
+
+    int IGNITE_CALL IgniteTransactionsCommit(gcj::JniContext* ctx, void* obj, long long id) {
+        return ctx->TransactionsCommit(static_cast<jobject>(obj), id);
+    }
+
+    void IGNITE_CALL IgniteTransactionsCommitAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId) {
+        return ctx->TransactionsCommitAsync(static_cast<jobject>(obj), id, futId);
+    }
+
+    int IGNITE_CALL IgniteTransactionsRollback(gcj::JniContext* ctx, void* obj, long long id) {
+        return ctx->TransactionsRollback(static_cast<jobject>(obj), id);
+    }
+
+    void IGNITE_CALL IgniteTransactionsRollbackAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId) {
+        return ctx->TransactionsRollbackAsync(static_cast<jobject>(obj), id, futId);
+    }
+
+    int IGNITE_CALL IgniteTransactionsClose(gcj::JniContext* ctx, void* obj, long long id) {
+        return ctx->TransactionsClose(static_cast<jobject>(obj), id);
+    }
+
+    int IGNITE_CALL IgniteTransactionsState(gcj::JniContext* ctx, void* obj, long long id) {
+        return ctx->TransactionsState(static_cast<jobject>(obj), id);
+    }
+
+    bool IGNITE_CALL IgniteTransactionsSetRollbackOnly(gcj::JniContext* ctx, void* obj, long long id) {
+        return ctx->TransactionsSetRollbackOnly(static_cast<jobject>(obj), id);
+    }
+
+    void IGNITE_CALL IgniteTransactionsResetMetrics(gcj::JniContext* ctx, void* obj) {
+        ctx->TransactionsResetMetrics(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteAcquire(gcj::JniContext* ctx, void* obj) {
+        return ctx->Acquire(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteRelease(void* obj) {
+        gcj::JniContext::Release(static_cast<jobject>(obj));
+    }
+
+    void IGNITE_CALL IgniteThrowToJava(gcj::JniContext* ctx, char* err) {
+        ctx->ThrowToJava(err);
+    }
+    
+    int IGNITE_CALL IgniteHandlersSize() {
+        return sizeof(gcj::JniHandlers);
+    }
+
+    void* IGNITE_CALL IgniteCreateContext(char** opts, int optsLen, gcj::JniHandlers* cbs) {
+        return gcj::JniContext::Create(opts, optsLen, *cbs);
+    }
+
+    void IGNITE_CALL IgniteDeleteContext(gcj::JniContext* ctx) {
+        delete ctx;
+    }
+
+    void IGNITE_CALL IgniteDestroyJvm(gcj::JniContext* ctx) {
+        ctx->DestroyJvm();
+    }
+
+    void* IGNITE_CALL IgniteEventsWithAsync(gcj::JniContext* ctx, void* obj) {
+        return ctx->EventsWithAsync(static_cast<jobject>(obj));
+    }
+
+    bool IGNITE_CALL IgniteEventsStopLocalListen(gcj::JniContext* ctx, void* obj, long long hnd) {
+        return ctx->EventsStopLocalListen(static_cast<jobject>(obj), hnd);
+    }
+
+    void IGNITE_CALL IgniteEventsLocalListen(gcj::JniContext* ctx, void* obj, long long hnd, int type) {
+        ctx->EventsLocalListen(static_cast<jobject>(obj), hnd, type);
+    }
+
+    bool IGNITE_CALL IgniteEventsIsEnabled(gcj::JniContext* ctx, void* obj, int type) {
+        return ctx->EventsIsEnabled(static_cast<jobject>(obj), type);
+    }    
+    
+	void* IGNITE_CALL IgniteServicesWithAsync(gcj::JniContext* ctx, void* obj) {
+		return ctx->ServicesWithAsync(static_cast<jobject>(obj));
+    }
+
+    void* IGNITE_CALL IgniteServicesWithServerKeepPortable(gcj::JniContext* ctx, void* obj) {
+    		return ctx->ServicesWithServerKeepPortable(static_cast<jobject>(obj));
+        }
+
+	void IGNITE_CALL IgniteServicesCancel(gcj::JniContext* ctx, void* obj, char* name) {
+		ctx->ServicesCancel(static_cast<jobject>(obj), name);
+    }
+
+	void IGNITE_CALL IgniteServicesCancelAll(gcj::JniContext* ctx, void* obj) {
+		ctx->ServicesCancelAll(static_cast<jobject>(obj));
+    }
+
+	void* IGNITE_CALL IgniteServicesGetServiceProxy(gcj::JniContext* ctx, void* obj, char* name, bool sticky) {
+		return ctx->ServicesGetServiceProxy(static_cast<jobject>(obj), name, sticky);
+    }
+}
\ No newline at end of file


[27/29] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: 6731627eb505455c65be9fc2ef4f39a4ca20cec3
Parents: 88a907c 66504a0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 1 17:38:09 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 1 17:38:09 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |   4 +-
 .../processors/platform/PlatformException.java  |  71 +++++++++++
 .../platform/PlatformExtendedException.java     |  57 +++++++++
 .../platform/PlatformNativeException.java       |  77 +++++++++++
 .../platform/PlatformNoCallbackException.java   |  50 ++++++++
 .../callback/PlatformCallbackGateway.java       |  10 +-
 .../internal/util/GridStripedSpinBusyLock.java  | 127 +++++++++++++++++++
 .../platform/PlatformContextImpl.java           |   4 +-
 .../processors/platform/PlatformException.java  |  71 -----------
 .../platform/PlatformExtendedException.java     |  39 ------
 .../platform/PlatformNoCallbackException.java   |  50 --------
 .../platform/cache/PlatformCache.java           |   2 +-
 .../PlatformCachePartialUpdateException.java    |  15 +--
 .../platform/compute/PlatformAbstractTask.java  |   1 +
 .../compute/PlatformNativeException.java        |  77 -----------
 .../platform/utils/PlatformUtils.java           |   2 +-
 16 files changed, 394 insertions(+), 263 deletions(-)
----------------------------------------------------------------------



[02/29] ignite git commit: updated code style xml file for Intellij IDEA

Posted by ak...@apache.org.
updated code style xml file for Intellij IDEA


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

Branch: refs/heads/ignite-843
Commit: 8438d0afdf411613536bf1448dbb5f72e98eccb7
Parents: 9e42600
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Sep 1 09:18:00 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Sep 1 09:18:00 2015 +0300

----------------------------------------------------------------------
 idea/ignite_codeStyle.xml | 11 ++---------
 1 file changed, 2 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8438d0af/idea/ignite_codeStyle.xml
----------------------------------------------------------------------
diff --git a/idea/ignite_codeStyle.xml b/idea/ignite_codeStyle.xml
index 14faf54..426588a 100644
--- a/idea/ignite_codeStyle.xml
+++ b/idea/ignite_codeStyle.xml
@@ -14,22 +14,15 @@
     </value>
   </option>
   <option name="PREFER_LONGER_NAMES" value="false" />
-  <option name="USE_SINGLE_CLASS_IMPORTS" value="false" />
-  <option name="CLASS_COUNT_TO_USE_IMPORT_ON_DEMAND" value="0" />
-  <option name="NAMES_COUNT_TO_USE_IMPORT_ON_DEMAND" value="0" />
+  <option name="CLASS_COUNT_TO_USE_IMPORT_ON_DEMAND" value="1000" />
+  <option name="NAMES_COUNT_TO_USE_IMPORT_ON_DEMAND" value="1000" />
   <option name="PACKAGES_TO_USE_IMPORT_ON_DEMAND">
     <value />
   </option>
   <option name="IMPORT_LAYOUT_TABLE">
     <value>
-      <package name="org.apache" withSubpackages="true" static="false" />
-      <package name="import org.gridgain" withSubpackages="true" static="false" />
-      <emptyLine />
       <package name="" withSubpackages="true" static="false" />
       <emptyLine />
-      <package name="javax" withSubpackages="true" static="false" />
-      <package name="java" withSubpackages="true" static="false" />
-      <emptyLine />
       <package name="" withSubpackages="true" static="true" />
     </value>
   </option>


[28/29] ignite git commit: Exclude test archives and project files from RAT

Posted by ak...@apache.org.
Exclude test archives and project files from RAT


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

Branch: refs/heads/ignite-843
Commit: 5a948dcb6b6697746bf9780ad0e673bef336480a
Parents: 6731627
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Sep 1 17:53:06 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Sep 1 17:53:06 2015 -0700

----------------------------------------------------------------------
 parent/pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5a948dcb/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 3556671..472d4ff 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -738,6 +738,10 @@
                                         <exclude>src/main/dotnet/Apache.Ignite.sln</exclude>
                                         <exclude>src/main/dotnet/Apache.Ignite.sln.DotSettings</exclude>
                                         <exclude>src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj</exclude>
+                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test1/1.1/test1-1.1.jar</exclude>
+                                        <exclude>src/test/portables/repo/org/apache/ignite/portable/test2/1.1/test2-1.1.jar</exclude>
+                                        <exclude>src/main/cpp/common/project/vs/common.vcxproj</exclude>
+                                        <exclude>src/main/cpp/common/project/vs/common.vcxproj.filters</exclude>
                                         <exclude>**/Makefile.am</exclude>
                                         <exclude>**/configure.ac</exclude>
                                         <exclude>**/*.vcsproj</exclude>


[05/29] ignite git commit: Fixed compilation issue in Ignite.

Posted by ak...@apache.org.
Fixed compilation issue in Ignite.


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

Branch: refs/heads/ignite-843
Commit: 16db3306ca1579e51eb42cb5efde7e9a9a6cf733
Parents: dfc1e40
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 10:30:05 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 10:30:05 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16db3306/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index f246107..bd24e51 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -29,7 +29,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheVariableTopologySelf
 import org.apache.ignite.internal.processors.cache.IgniteAtomicCacheEntryProcessorNodeJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorNodeJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
-import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop
+import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionNotLoadedEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedAffinityFilterSelfTest;


[18/29] ignite git commit: gg-10705

Posted by ak...@apache.org.
gg-10705


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

Branch: refs/heads/ignite-843
Commit: fda434c89eaf19ce701cd54ad14d037cea9b7c1c
Parents: 0e25f55
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Sep 1 13:20:31 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Sep 1 13:20:31 2015 +0300

----------------------------------------------------------------------
 modules/core/pom.xml | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fda434c8/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 6f1cbf8..698606e 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -40,6 +40,10 @@
         </repository>
     </repositories>
 
+    <properties>
+        <ignite.update.notifier.product>apache-ignite</ignite.update.notifier.product>
+    </properties>
+
     <dependencies>
         <dependency>
             <groupId>javax.cache</groupId>
@@ -310,7 +314,7 @@
 
                                         <replaceregexp file="${props.file}" byline="true">
                                             <regexp pattern="ignite.update.status.params=.*" />
-                                            <substitution expression="ignite.update.status.params=ver=${project.version}" />
+                                            <substitution expression="ignite.update.status.params=ver=${project.version}&amp;product=${ignite.update.notifier.product}" />
                                         </replaceregexp>
 
                                         <replaceregexp file="${props.file}" byline="true">


[20/29] ignite git commit: IGNITE-1343: Implemented PlatformContext interface.

Posted by ak...@apache.org.
IGNITE-1343: Implemented PlatformContext interface.


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

Branch: refs/heads/ignite-843
Commit: 5f58bbb570a1a2f8939f87bc2d9696f4139ddf20
Parents: 2cd3da2
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 15:45:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 15:45:21 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformContextImpl.java           | 623 +++++++++++++++++++
 .../messaging/PlatformMessageFilterImpl.java    |   2 +-
 2 files changed, 624 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f58bbb5/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
new file mode 100644
index 0000000..6d1d7a7
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -0,0 +1,623 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.CacheEvent;
+import org.apache.ignite.events.CacheQueryExecutedEvent;
+import org.apache.ignite.events.CacheQueryReadEvent;
+import org.apache.ignite.events.CacheRebalancingEvent;
+import org.apache.ignite.events.CheckpointEvent;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventAdapter;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.events.JobEvent;
+import org.apache.ignite.events.SwapSpaceEvent;
+import org.apache.ignite.events.TaskEvent;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableMetaDataImpl;
+import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessorImpl;
+import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
+import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter;
+import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl;
+import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryRemoteFilter;
+import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
+import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilter;
+import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilterImpl;
+import org.apache.ignite.internal.processors.platform.compute.PlatformAbstractTask;
+import org.apache.ignite.internal.processors.platform.compute.PlatformClosureJob;
+import org.apache.ignite.internal.processors.platform.compute.PlatformFullJob;
+import org.apache.ignite.internal.processors.platform.compute.PlatformJob;
+import org.apache.ignite.internal.processors.platform.compute.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver;
+import org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl;
+import org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl;
+import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManager;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
+import org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl;
+import org.apache.ignite.internal.processors.platform.utils.PlatformReaderBiClosure;
+import org.apache.ignite.internal.processors.platform.utils.PlatformReaderClosure;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T4;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.portable.PortableMetadata;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Implementation of platform context.
+ */
+public class PlatformContextImpl implements PlatformContext {
+    /** Supported event types. */
+    private static final Set<Integer> evtTyps;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Marshaller. */
+    private final GridPortableMarshaller marsh;
+
+    /** Memory manager. */
+    private final PlatformMemoryManagerImpl mem;
+
+    /** Callback gateway. */
+    private final PlatformCallbackGateway gate;
+
+    /** Cache object processor. */
+    private final CacheObjectPortableProcessorImpl cacheObjProc;
+
+    /** Node ids that has been sent to native platform. */
+    private final Set<UUID> sentNodes = Collections.newSetFromMap(new ConcurrentHashMap<UUID, Boolean>());
+
+    /**
+     * Static initializer.
+     */
+    static {
+        Set<Integer> evtTyps0 = new HashSet<>();
+
+        addEventTypes(evtTyps0, EventType.EVTS_CACHE);
+        addEventTypes(evtTyps0, EventType.EVTS_CACHE_QUERY);
+        addEventTypes(evtTyps0, EventType.EVTS_CACHE_REBALANCE);
+        addEventTypes(evtTyps0, EventType.EVTS_CHECKPOINT);
+        addEventTypes(evtTyps0, EventType.EVTS_DISCOVERY_ALL);
+        addEventTypes(evtTyps0, EventType.EVTS_JOB_EXECUTION);
+        addEventTypes(evtTyps0, EventType.EVTS_SWAPSPACE);
+        addEventTypes(evtTyps0, EventType.EVTS_TASK_EXECUTION);
+
+        evtTyps = Collections.unmodifiableSet(evtTyps0);
+    }
+
+    /**
+     * Adds all elements to a set.
+     * @param set Set.
+     * @param items Items.
+     */
+    private static void addEventTypes(Set<Integer> set, int[] items) {
+        for (int i : items)
+            set.add(i);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     * @param gate Callback gateway.
+     * @param mem Memory manager.
+     */
+    public PlatformContextImpl(GridKernalContext ctx, PlatformCallbackGateway gate, PlatformMemoryManagerImpl mem) {
+        this.ctx = ctx;
+        this.gate = gate;
+        this.mem = mem;
+
+        cacheObjProc = (CacheObjectPortableProcessorImpl)ctx.cacheObjects();
+
+        marsh = cacheObjProc.marshaller();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridKernalContext kernalContext() {
+        return ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformMemoryManager memory() {
+        return mem;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformCallbackGateway gateway() {
+        return gate;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableRawReaderEx reader(PlatformMemory mem) {
+        return reader(mem.input());
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableRawReaderEx reader(PlatformInputStream in) {
+        return marsh.reader(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableRawWriterEx writer(PlatformMemory mem) {
+        return writer(mem.output());
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableRawWriterEx writer(PlatformOutputStream out) {
+        return marsh.writer(out);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addNode(ClusterNode node) {
+        if (node == null || sentNodes.contains(node.id()))
+            return;
+
+        // Send node info to the native platform
+        try (PlatformMemory mem0 = mem.allocate()) {
+            PlatformOutputStream out = mem0.output();
+
+            PortableRawWriterEx w = writer(out);
+
+            w.writeUuid(node.id());
+
+            Map<String, Object> attrs = new HashMap<>(node.attributes());
+
+            Iterator<Map.Entry<String, Object>> attrIter = attrs.entrySet().iterator();
+
+            while (attrIter.hasNext()) {
+                Map.Entry<String, Object> entry = attrIter.next();
+
+                Object val = entry.getValue();
+
+                if (val != null && !val.getClass().getName().startsWith("java.lang"))
+                    attrIter.remove();
+            }
+
+            w.writeMap(attrs);
+            w.writeCollection(node.addresses());
+            w.writeCollection(node.hostNames());
+            w.writeLong(node.order());
+            w.writeBoolean(node.isLocal());
+            w.writeBoolean(node.isDaemon());
+            writeClusterMetrics(w, node.metrics());
+
+            out.synchronize();
+
+            gateway().nodeInfo(mem0.pointer());
+        }
+
+        sentNodes.add(node.id());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeNode(PortableRawWriterEx writer, ClusterNode node) {
+        if (node == null) {
+            writer.writeUuid(null);
+
+            return;
+        }
+
+        addNode(node);
+
+        writer.writeUuid(node.id());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeNodes(PortableRawWriterEx writer, Collection<ClusterNode> nodes) {
+        if (nodes == null) {
+            writer.writeInt(-1);
+
+            return;
+        }
+
+        writer.writeInt(nodes.size());
+
+        for (ClusterNode n : nodes) {
+            addNode(n);
+
+            writer.writeUuid(n.id());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeClusterMetrics(PortableRawWriterEx writer, @Nullable ClusterMetrics metrics) {
+        if (metrics == null)
+            writer.writeBoolean(false);
+        else {
+            writer.writeBoolean(true);
+
+            writer.writeLong(metrics.getLastUpdateTime());
+            writer.writeDate(new Date(metrics.getLastUpdateTime()));
+            writer.writeInt(metrics.getMaximumActiveJobs());
+            writer.writeInt(metrics.getCurrentActiveJobs());
+            writer.writeFloat(metrics.getAverageActiveJobs());
+            writer.writeInt(metrics.getMaximumWaitingJobs());
+
+            writer.writeInt(metrics.getCurrentWaitingJobs());
+            writer.writeFloat(metrics.getAverageWaitingJobs());
+            writer.writeInt(metrics.getMaximumRejectedJobs());
+            writer.writeInt(metrics.getCurrentRejectedJobs());
+            writer.writeFloat(metrics.getAverageRejectedJobs());
+
+            writer.writeInt(metrics.getTotalRejectedJobs());
+            writer.writeInt(metrics.getMaximumCancelledJobs());
+            writer.writeInt(metrics.getCurrentCancelledJobs());
+            writer.writeFloat(metrics.getAverageCancelledJobs());
+            writer.writeInt(metrics.getTotalCancelledJobs());
+
+            writer.writeInt(metrics.getTotalExecutedJobs());
+            writer.writeLong(metrics.getMaximumJobWaitTime());
+            writer.writeLong(metrics.getCurrentJobWaitTime());
+            writer.writeDouble(metrics.getAverageJobWaitTime());
+            writer.writeLong(metrics.getMaximumJobExecuteTime());
+
+            writer.writeLong(metrics.getCurrentJobExecuteTime());
+            writer.writeDouble(metrics.getAverageJobExecuteTime());
+            writer.writeInt(metrics.getTotalExecutedTasks());
+            writer.writeLong(metrics.getTotalIdleTime());
+            writer.writeLong(metrics.getCurrentIdleTime());
+
+            writer.writeInt(metrics.getTotalCpus());
+            writer.writeDouble(metrics.getCurrentCpuLoad());
+            writer.writeDouble(metrics.getAverageCpuLoad());
+            writer.writeDouble(metrics.getCurrentGcCpuLoad());
+            writer.writeLong(metrics.getHeapMemoryInitialized());
+
+            writer.writeLong(metrics.getHeapMemoryUsed());
+            writer.writeLong(metrics.getHeapMemoryCommitted());
+            writer.writeLong(metrics.getHeapMemoryMaximum());
+            writer.writeLong(metrics.getHeapMemoryTotal());
+            writer.writeLong(metrics.getNonHeapMemoryInitialized());
+
+            writer.writeLong(metrics.getNonHeapMemoryUsed());
+            writer.writeLong(metrics.getNonHeapMemoryCommitted());
+            writer.writeLong(metrics.getNonHeapMemoryMaximum());
+            writer.writeLong(metrics.getNonHeapMemoryTotal());
+            writer.writeLong(metrics.getUpTime());
+
+            writer.writeDate(new Date(metrics.getStartTime()));
+            writer.writeDate(new Date(metrics.getNodeStartTime()));
+            writer.writeInt(metrics.getCurrentThreadCount());
+            writer.writeInt(metrics.getMaximumThreadCount());
+            writer.writeLong(metrics.getTotalStartedThreadCount());
+
+            writer.writeInt(metrics.getCurrentDaemonThreadCount());
+            writer.writeLong(metrics.getLastDataVersion());
+            writer.writeInt(metrics.getSentMessagesCount());
+            writer.writeLong(metrics.getSentBytesCount());
+            writer.writeInt(metrics.getReceivedMessagesCount());
+
+            writer.writeLong(metrics.getReceivedBytesCount());
+            writer.writeInt(metrics.getOutboundMessagesQueueSize());
+
+            writer.writeInt(metrics.getTotalNodes());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override public void processMetadata(PortableRawReaderEx reader) {
+        Collection<T4<Integer, String, String, Map<String, Integer>>> metas = PlatformUtils.readCollection(reader,
+            new PlatformReaderClosure<T4<Integer, String, String, Map<String, Integer>>>() {
+                @Override public T4<Integer, String, String, Map<String, Integer>> read(PortableRawReaderEx reader) {
+                    int typeId = reader.readInt();
+                    String typeName = reader.readString();
+                    String affKey = reader.readString();
+
+                    Map<String, Integer> fields = PlatformUtils.readMap(reader,
+                        new PlatformReaderBiClosure<String, Integer>() {
+                            @Override public IgniteBiTuple<String, Integer> read(PortableRawReaderEx reader) {
+                                return F.t(reader.readString(), reader.readInt());
+                            }
+                        });
+
+                    return new T4<>(typeId, typeName, affKey, fields);
+                }
+            }
+        );
+
+        for (T4<Integer, String, String, Map<String, Integer>> meta : metas)
+            cacheObjProc.updateMetaData(meta.get1(), meta.get2(), meta.get3(), meta.get4());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeMetadata(PortableRawWriterEx writer, int typeId) {
+        writeMetadata0(writer, typeId, cacheObjProc.metadata(typeId));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeAllMetadata(PortableRawWriterEx writer) {
+        Collection<PortableMetadata> metas = cacheObjProc.metadata();
+
+        writer.writeInt(metas.size());
+
+        for (org.apache.ignite.portable.PortableMetadata m : metas)
+            writeMetadata0(writer, cacheObjProc.typeId(m.typeName()), m);
+    }
+
+    /**
+     * Write portable metadata.
+     *
+     * @param writer Writer.
+     * @param typeId Type id.
+     * @param meta Metadata.
+     */
+    private void writeMetadata0(PortableRawWriterEx writer, int typeId, PortableMetadata meta) {
+        if (meta == null)
+            writer.writeBoolean(false);
+        else {
+            writer.writeBoolean(true);
+
+            Map<String, String> metaFields = ((PortableMetaDataImpl)meta).fields0();
+
+            Map<String, Integer> fields = U.newHashMap(metaFields.size());
+
+            for (Map.Entry<String, String> metaField : metaFields.entrySet())
+                fields.put(metaField.getKey(), CacheObjectPortableProcessorImpl.fieldTypeId(metaField.getValue()));
+
+            writer.writeInt(typeId);
+            writer.writeString(meta.typeName());
+            writer.writeString(meta.affinityKeyFieldName());
+            writer.writeMap(fields);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformContinuousQuery createContinuousQuery(long ptr, boolean hasFilter,
+        @Nullable Object filter) {
+        return new PlatformContinuousQueryImpl(this, ptr, hasFilter, filter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformContinuousQueryFilter createContinuousQueryFilter(Object filter) {
+        return new PlatformContinuousQueryRemoteFilter(filter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformMessageFilter createRemoteMessageFilter(Object filter, long ptr) {
+        return new PlatformMessageFilterImpl(filter, ptr, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isEventTypeSupported(int evtTyp) {
+        return evtTyps.contains(evtTyp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeEvent(PortableRawWriterEx writer, Event evt) {
+        assert writer != null;
+
+        if (evt == null)
+        {
+            writer.writeInt(-1);
+
+            return;
+        }
+
+        EventAdapter evt0 = (EventAdapter)evt;
+
+        if (evt0 instanceof CacheEvent) {
+            writer.writeInt(2);
+            writeCommonEventData(writer, evt0);
+
+            CacheEvent event0 = (CacheEvent)evt0;
+
+            writer.writeString(event0.cacheName());
+            writer.writeInt(event0.partition());
+            writer.writeBoolean(event0.isNear());
+            writeNode(writer, event0.eventNode());
+            writer.writeObject(event0.key());
+            PlatformUtils.writeIgniteUuid(writer, event0.xid());
+            writer.writeObject(event0.lockId());
+            writer.writeObject(event0.newValue());
+            writer.writeObject(event0.oldValue());
+            writer.writeBoolean(event0.hasOldValue());
+            writer.writeBoolean(event0.hasNewValue());
+            writer.writeUuid(event0.subjectId());
+            writer.writeString(event0.closureClassName());
+            writer.writeString(event0.taskName());
+        }
+        else if (evt0 instanceof CacheQueryExecutedEvent) {
+            writer.writeInt(3);
+            writeCommonEventData(writer, evt0);
+
+            CacheQueryExecutedEvent event0 = (CacheQueryExecutedEvent)evt0;
+
+            writer.writeString(event0.queryType());
+            writer.writeString(event0.cacheName());
+            writer.writeString(event0.className());
+            writer.writeString(event0.clause());
+            writer.writeUuid(event0.subjectId());
+            writer.writeString(event0.taskName());
+        }
+        else if (evt0 instanceof CacheQueryReadEvent) {
+            writer.writeInt(4);
+            writeCommonEventData(writer, evt0);
+
+            CacheQueryReadEvent event0 = (CacheQueryReadEvent)evt0;
+
+            writer.writeString(event0.queryType());
+            writer.writeString(event0.cacheName());
+            writer.writeString(event0.className());
+            writer.writeString(event0.clause());
+            writer.writeUuid(event0.subjectId());
+            writer.writeString(event0.taskName());
+            writer.writeObject(event0.key());
+            writer.writeObject(event0.value());
+            writer.writeObject(event0.oldValue());
+            writer.writeObject(event0.row());
+        }
+        else if (evt0 instanceof CacheRebalancingEvent) {
+            writer.writeInt(5);
+            writeCommonEventData(writer, evt0);
+
+            CacheRebalancingEvent event0 = (CacheRebalancingEvent)evt0;
+
+            writer.writeString(event0.cacheName());
+            writer.writeInt(event0.partition());
+            writeNode(writer, event0.discoveryNode());
+            writer.writeInt(event0.discoveryEventType());
+            writer.writeString(event0.discoveryEventName());
+            writer.writeLong(event0.discoveryTimestamp());
+        }
+        else if (evt0 instanceof CheckpointEvent) {
+            writer.writeInt(6);
+            writeCommonEventData(writer, evt0);
+
+            CheckpointEvent event0 = (CheckpointEvent)evt0;
+
+            writer.writeString(event0.key());
+        }
+        else if (evt0 instanceof DiscoveryEvent) {
+            writer.writeInt(7);
+            writeCommonEventData(writer, evt0);
+
+            DiscoveryEvent event0 = (DiscoveryEvent)evt0;
+
+            writeNode(writer, event0.eventNode());
+            writer.writeLong(event0.topologyVersion());
+
+            writeNodes(writer, event0.topologyNodes());
+        }
+        else if (evt0 instanceof JobEvent) {
+            writer.writeInt(8);
+            writeCommonEventData(writer, evt0);
+
+            JobEvent event0 = (JobEvent)evt0;
+
+            writer.writeString(event0.taskName());
+            writer.writeString(event0.taskClassName());
+            PlatformUtils.writeIgniteUuid(writer, event0.taskSessionId());
+            PlatformUtils.writeIgniteUuid(writer, event0.jobId());
+            writeNode(writer, event0.taskNode());
+            writer.writeUuid(event0.taskSubjectId());
+        }
+        else if (evt0 instanceof SwapSpaceEvent) {
+            writer.writeInt(9);
+            writeCommonEventData(writer, evt0);
+
+            SwapSpaceEvent event0 = (SwapSpaceEvent)evt0;
+
+            writer.writeString(event0.space());
+        }
+        else if (evt0 instanceof TaskEvent) {
+            writer.writeInt(10);
+            writeCommonEventData(writer, evt0);
+
+            TaskEvent event0 = (TaskEvent)evt0;
+
+            writer.writeString(event0.taskName());
+            writer.writeString(event0.taskClassName());
+            PlatformUtils.writeIgniteUuid(writer, event0.taskSessionId());
+            writer.writeBoolean(event0.internal());
+            writer.writeUuid(event0.subjectId());
+        }
+        else
+            throw new IgniteException("Unsupported event: " + evt);
+    }
+
+    /**
+     * Write common event data.
+     *
+     * @param writer Writer.
+     * @param evt Event.
+     */
+    private void writeCommonEventData(PortableRawWriterEx writer, EventAdapter evt) {
+        PlatformUtils.writeIgniteUuid(writer, evt.id());
+        writer.writeLong(evt.localOrder());
+        writeNode(writer, evt.node());
+        writer.writeString(evt.message());
+        writer.writeInt(evt.type());
+        writer.writeString(evt.name());
+        writer.writeDate(new Date(evt.timestamp()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformEventFilterListener createLocalEventFilter(long hnd) {
+        return new PlatformEventFilterListenerImpl(hnd, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformEventFilterListener createRemoteEventFilter(Object pred, int... types) {
+        return new PlatformEventFilterListenerImpl(pred, types);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCheckedException createNativeException(Object cause) {
+        return new PlatformNativeException(cause);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformJob createJob(Object task, long ptr, @Nullable Object job) {
+        return new PlatformFullJob(this, (PlatformAbstractTask)task, ptr, job);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformJob createClosureJob(Object task, long ptr, Object job) {
+        return new PlatformClosureJob((PlatformAbstractTask)task, ptr, job);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformCacheEntryProcessor createCacheEntryProcessor(Object proc, long ptr) {
+        return new PlatformCacheEntryProcessorImpl(proc, ptr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformCacheEntryFilter createCacheEntryFilter(Object filter, long ptr) {
+        return new PlatformCacheEntryFilterImpl(filter, ptr, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformStreamReceiver createStreamReceiver(Object rcv, long ptr, boolean keepPortable) {
+        return new PlatformStreamReceiverImpl(rcv, ptr, keepPortable, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformClusterNodeFilter createClusterNodeFilter(Object filter) {
+        return new PlatformClusterNodeFilterImpl(filter, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f58bbb5/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
index 1e42914..67d5bbb 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
@@ -50,7 +50,7 @@ public class PlatformMessageFilterImpl extends PlatformAbstractPredicate impleme
      * @param ptr Pointer to predicate in the native platform.
      * @param ctx Kernal context.
      */
-    protected PlatformMessageFilterImpl(Object pred, long ptr, PlatformContext ctx) {
+    public PlatformMessageFilterImpl(Object pred, long ptr, PlatformContext ctx) {
         super(pred, ptr, ctx);
     }
 


[23/29] ignite git commit: IGNITE-1346: Field rename.

Posted by ak...@apache.org.
IGNITE-1346: Field rename.


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

Branch: refs/heads/ignite-843
Commit: 34ce97b685de9097303ee68d380b7a3d9b43348a
Parents: aec0631
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 16:40:35 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 16:40:35 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/GridStripedSpinBusyLock.java     | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/34ce97b6/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
index 678d521..614a88f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
@@ -31,16 +31,16 @@ public class GridStripedSpinBusyLock {
     /** Default amount of stripes. */
     private static final int DFLT_STRIPE_CNT = Runtime.getRuntime().availableProcessors() * 4;
 
-    /** States; they are not subjects to false-sharing because actual values are located far from each other. */
-    private final AtomicInteger[] states;
-
     /** Thread index. */
-    private static ThreadLocal<Integer> threadIdx = new ThreadLocal<Integer>() {
+    private static ThreadLocal<Integer> THREAD_IDX = new ThreadLocal<Integer>() {
         @Override protected Integer initialValue() {
             return new Random().nextInt(Integer.MAX_VALUE);
         }
     };
 
+    /** States; they are not subjects to false-sharing because actual values are located far from each other. */
+    private final AtomicInteger[] states;
+
     /**
      * Default constructor.
      */
@@ -120,6 +120,6 @@ public class GridStripedSpinBusyLock {
      * @return State.
      */
     private AtomicInteger state() {
-        return states[threadIdx.get() % states.length];
+        return states[THREAD_IDX.get() % states.length];
     }
 }


[10/29] ignite git commit: IGNITE-1337: Moved "common" CPP project to Ignite.

Posted by ak...@apache.org.
IGNITE-1337: Moved "common" CPP project to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 7ca8cbc774b0c23aca537b078898707a2ede3cbd
Parents: 10e51d1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 10:39:17 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 10:39:17 2015 +0300

----------------------------------------------------------------------
 .../platform/src/main/cpp/common/Makefile.am    |   45 +
 .../platform/src/main/cpp/common/configure.ac   |   62 +
 .../src/main/cpp/common/ignite-common.pc.in     |    9 +
 .../src/main/cpp/common/include/Makefile.am     |   22 +
 .../common/include/ignite/common/concurrent.h   |  210 ++
 .../cpp/common/include/ignite/common/exports.h  |  145 ++
 .../cpp/common/include/ignite/common/java.h     |  652 ++++++
 .../cpp/common/os/linux/include/Makefile.am     |   21 +
 .../os/linux/include/ignite/common/common.h     |   81 +
 .../linux/include/ignite/common/concurrent_os.h |  394 ++++
 .../src/main/cpp/common/os/linux/src/common.cpp |   59 +
 .../cpp/common/os/linux/src/concurrent_os.cpp   |  175 ++
 .../os/win/include/ignite/common/common.h       |   56 +
 .../win/include/ignite/common/concurrent_os.h   |  406 ++++
 .../src/main/cpp/common/os/win/src/common.cpp   |   65 +
 .../cpp/common/os/win/src/concurrent_os.cpp     |  151 ++
 .../src/main/cpp/common/project/README.TXT      |    1 +
 .../src/main/cpp/common/project/vs/README.TXT   |    1 +
 .../main/cpp/common/project/vs/common.vcxproj   |  202 ++
 .../common/project/vs/common.vcxproj.filters    |   54 +
 .../src/main/cpp/common/project/vs/module.def   |   99 +
 .../src/main/cpp/common/project/vs/targetver.h  |   25 +
 .../src/main/cpp/common/src/concurrent.cpp      |   94 +
 .../src/main/cpp/common/src/exports.cpp         |  413 ++++
 .../platform/src/main/cpp/common/src/java.cpp   | 2204 ++++++++++++++++++
 parent/pom.xml                                  |    5 +
 26 files changed, 5651 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/Makefile.am b/modules/platform/src/main/cpp/common/Makefile.am
new file mode 100644
index 0000000..f5ca5dd
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/Makefile.am
@@ -0,0 +1,45 @@
+##
+## 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.
+##
+
+ACLOCAL_AMFLAGS = "-Im4"
+
+SUBDIRS = . include os/linux/include
+DIST_SUBDIRS = . include os/linux/include
+
+AM_CPPFLAGS = -I$(srcdir)/include -I$(srcdir)/os/linux/include -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -DIGNITE_IMPL
+AM_CXXFLAGS = -Wall -std=c++0x
+LIB_LDFLAGS = -no-undefined -version-info 1
+
+COMMON_SRC = os/linux/src/concurrent_os.cpp \
+             src/concurrent.cpp \
+             src/java.cpp \
+             src/exports.cpp \
+             os/linux/src/common.cpp
+
+lib_LTLIBRARIES = libignite-common.la
+libignite_common_la_SOURCES = $(COMMON_SRC)
+libignite_common_la_LIBADD = -L$(JAVA_HOME)/jre/lib/amd64/server
+libignite_common_la_LDFLAGS = $(LIB_LDFLAGS) -L/usr/local/lib -ljvm -version-info 0:0:0 -release $(PACKAGE_VERSION)
+
+pkgconfigdir = $(libdir)/pkgconfig
+pkgconfig_DATA = ignite-common.pc
+
+clean-local:
+	$(RM) *.gcno *.gcda
+
+clean-docs:
+	$(RM) $(DX_CLEANFILES)

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/configure.ac b/modules/platform/src/main/cpp/common/configure.ac
new file mode 100644
index 0000000..7706737
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/configure.ac
@@ -0,0 +1,62 @@
+#
+# 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.
+#
+
+#                                               -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+AC_PREREQ([2.69])
+AC_INIT([Ignite JNI bridge for C++], [7.4.1], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
+AC_CONFIG_SRCDIR(src)
+
+AC_CANONICAL_SYSTEM
+AC_CONFIG_MACRO_DIR([m4])
+AC_LANG([C++])
+
+# Initialize automake
+AM_INIT_AUTOMAKE([-Wall foreign subdir-objects])
+AC_CONFIG_HEADER(config.h)
+
+AM_PROG_AR
+
+# Checks for programs.
+GXX="-g -O2"
+
+AC_PROG_CXX
+
+# Initialize Libtool
+LT_INIT
+
+# Checks for libraries.
+AC_CHECK_LIB([pthread], [pthread_mutex_lock])
+
+# Checks for header files.
+
+# Checks for typedefs, structures, and compiler characteristics.
+AC_C_INLINE
+AC_TYPE_INT16_T
+AC_TYPE_INT32_T
+AC_TYPE_INT64_T
+AC_TYPE_INT8_T
+AC_TYPE_PID_T
+AC_TYPE_SIZE_T
+
+# Checks for library functions.
+AC_FUNC_ERROR_AT_LINE
+
+AC_CONFIG_FILES(Makefile include/Makefile os/linux/include/Makefile ignite-common.pc)
+
+AC_OUTPUT

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/ignite-common.pc.in
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/ignite-common.pc.in b/modules/platform/src/main/cpp/common/ignite-common.pc.in
new file mode 100644
index 0000000..3cd3cec
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/ignite-common.pc.in
@@ -0,0 +1,9 @@
+prefix=@prefix@
+exec_prefix=@exec_prefix@
+libdir=@libdir@
+includedir=@includedir@
+
+Name: ignite-common
+Description: Ignite JNI bridge for C++.
+Version: @PACKAGE_VERSION@
+Libs: -L${libdir} -lignite-common

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/include/Makefile.am b/modules/platform/src/main/cpp/common/include/Makefile.am
new file mode 100644
index 0000000..5db1d4a
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/include/Makefile.am
@@ -0,0 +1,22 @@
+##
+## 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.
+##
+
+ACLOCAL_AMFLAGS = "-Im4"
+
+nobase_include_HEADERS = ignite/common/concurrent.h \
+                         ignite/common/java.h \
+                         ignite/common/exports.h

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/include/ignite/common/concurrent.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/include/ignite/common/concurrent.h b/modules/platform/src/main/cpp/common/include/ignite/common/concurrent.h
new file mode 100644
index 0000000..1c9ab22
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/include/ignite/common/concurrent.h
@@ -0,0 +1,210 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_COMMON_CONCURRENT
+#define _IGNITE_COMMON_CONCURRENT
+
+#include "ignite/common/concurrent_os.h"
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace concurrent
+        {
+            /**
+             * Default deleter implementation.
+             *
+             * @param obj Object to be deleted.
+             */
+            template<typename T>
+            IGNITE_IMPORT_EXPORT void SharedPointerDefaultDeleter(T* obj)
+            {
+                delete obj;
+            }
+
+            /**
+             * Holder of shared pointer data.
+             */
+            class IGNITE_IMPORT_EXPORT SharedPointerImpl
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param ptr Raw pointer.
+                 */
+                SharedPointerImpl(void* ptr);
+
+                /**
+                 * Get raw pointer.
+                 *
+                 * @return Raw pointer.
+                 */
+                void* Pointer();
+
+                /**
+                 * Increment usage counter.
+                 */
+                void Increment();
+
+                /**
+                 * Decrement usage counter.
+                 *
+                 * @return True if counter reached zero.
+                 */
+                bool Decrement();
+            private:
+                /** Raw pointer. */
+                void* ptr;
+
+                /** Reference count. */
+                int32_t refCnt;
+
+                IGNITE_NO_COPY_ASSIGNMENT(SharedPointerImpl)
+            };
+
+            /**
+             * Shared pointer.
+             */
+            template<typename T>
+            class IGNITE_IMPORT_EXPORT SharedPointer
+            {
+            public:
+                /**
+                 * Constructor.
+                 */
+                SharedPointer() : impl(NULL), deleter(NULL)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Constructor.
+                 *
+                 * @param ptr Raw pointer.
+                 */
+                explicit SharedPointer(T* ptr)
+                {
+                    if (ptr)
+                    {
+                        impl = new SharedPointerImpl(ptr);
+                        deleter = SharedPointerDefaultDeleter;
+                    }
+                    else
+                    {
+                        impl = NULL;
+                        deleter = NULL;
+                    }
+                }
+
+                /**
+                 * Constructor.
+                 *
+                 * @param ptr Raw pointer.
+                 * @param deleter Delete function.
+                 */
+                SharedPointer(T* ptr, void(*deleter)(T*))
+                {
+                    if (ptr)
+                    {
+                        this->impl = new SharedPointerImpl(ptr);
+                        this->deleter = deleter;
+                    }
+                    else
+                    {
+                        this->impl = NULL;
+                        this->deleter = NULL;
+                    }
+                }
+
+                /**
+                 * Copy constructor.
+                 *
+                 * @param other Instance to copy.
+                 */
+                SharedPointer(const SharedPointer& other)
+                {
+                    impl = other.impl;
+                    deleter = other.deleter;
+
+                    if (impl)
+                        impl->Increment();
+                }
+
+                /**
+                 * Assignment operator.
+                 *
+                 * @param other Other instance.
+                 */
+                SharedPointer& operator=(const SharedPointer& other)
+                {
+                    if (this != &other)
+                    {
+                        // 1. Create new instance.
+                        SharedPointer tmp(other);
+
+                        // 2. Swap with temp.
+                        SharedPointerImpl* impl0 = impl;
+                        void(*deleter0)(T*) = deleter;
+
+                        impl = tmp.impl;
+                        deleter = tmp.deleter;
+
+                        tmp.impl = impl0;
+                        tmp.deleter = deleter0;
+                    }
+
+                    return *this;
+                }
+
+                /**
+                 * Destructor.
+                 */
+                ~SharedPointer()
+                {
+                    if (impl && impl->Decrement())
+                    {
+                        T* ptr = Get();
+
+                        delete impl;
+
+                        deleter(ptr);
+                    }
+                }
+
+                /**
+                 * Get raw pointer.
+                 *
+                 * @return Raw pointer.
+                 */
+                T* Get()
+                {
+                    return impl ? static_cast<T*>(impl->Pointer()) : NULL;
+                }
+            private:
+                /** Implementation. */
+                SharedPointerImpl* impl;
+
+                /** Delete function. */
+                void(*deleter)(T*);
+            };
+        }
+    }
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/include/ignite/common/exports.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/include/ignite/common/exports.h b/modules/platform/src/main/cpp/common/include/ignite/common/exports.h
new file mode 100644
index 0000000..48e86f0
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/include/ignite/common/exports.h
@@ -0,0 +1,145 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_COMMON_EXPORTS
+#define _IGNITE_COMMON_EXPORTS
+
+#include "ignite/common/java.h"
+
+namespace gcj = ignite::common::java;
+
+extern "C" {
+    int IGNITE_CALL IgniteReallocate(long long memPtr, int cap);
+
+    void* IGNITE_CALL IgniteIgnitionStart(gcj::JniContext* ctx, char* cfgPath, char* gridName, int factoryId, long long dataPtr);
+    void* IGNITE_CALL IgniteIgnitionInstance(gcj::JniContext* ctx, char* gridName);
+    long long IGNITE_CALL IgniteIgnitionEnvironmentPointer(gcj::JniContext* ctx, char* gridName);
+    bool IGNITE_CALL IgniteIgnitionStop(gcj::JniContext* ctx, char* gridName, bool cancel);
+    void IGNITE_CALL IgniteIgnitionStopAll(gcj::JniContext* ctx, bool cancel);
+
+    void IGNITE_CALL IgniteProcessorReleaseStart(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProcessorProjection(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProcessorCache(gcj::JniContext* ctx, void* obj, char* name);
+    void* IGNITE_CALL IgniteProcessorCreateCache(gcj::JniContext* ctx, void* obj, char* name);
+    void* IGNITE_CALL IgniteProcessorGetOrCreateCache(gcj::JniContext* ctx, void* obj, char* name);
+    void* IGNITE_CALL IgniteProcessorAffinity(gcj::JniContext* ctx, void* obj, char* name);
+    void* IGNITE_CALL IgniteProcessorDataStreamer(gcj::JniContext* ctx, void* obj, char* name, bool keepPortable);
+    void* IGNITE_CALL IgniteProcessorTransactions(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProcessorCompute(gcj::JniContext* ctx, void* obj, void* prj);
+    void* IGNITE_CALL IgniteProcessorMessage(gcj::JniContext* ctx, void* obj, void* prj);
+    void* IGNITE_CALL IgniteProcessorEvents(gcj::JniContext* ctx, void* obj, void* prj);
+    void* IGNITE_CALL IgniteProcessorServices(gcj::JniContext* ctx, void* obj, void* prj);
+    void* IGNITE_CALL IgniteProcessorExtensions(gcj::JniContext* ctx, void* obj);
+    
+    long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
+    void IGNITE_CALL IgniteTargetInStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, long long inMemPtr, long long outMemPtr);
+    void* IGNITE_CALL IgniteTargetInStreamOutObject(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
+    void IGNITE_CALL IgniteTargetInObjectStreamOutStream(gcj::JniContext* ctx, void* obj, int opType, void* arg, long long inMemPtr, long long outMemPtr);
+    long long IGNITE_CALL IgniteTargetOutLong(gcj::JniContext* ctx, void* obj, int opType);
+    void IGNITE_CALL IgniteTargetOutStream(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
+    void* IGNITE_CALL IgniteTargetOutObject(gcj::JniContext* ctx, void* obj, int opType);
+    void IGNITE_CALL IgniteTargetListenFuture(gcj::JniContext* ctx, void* obj, long long futId, int typ);
+    void IGNITE_CALL IgniteTargetListenFutureForOperation(gcj::JniContext* ctx, void* obj, long long futId, int typ, int opId);
+
+    int IGNITE_CALL IgniteAffinityPartitions(gcj::JniContext* ctx, void* obj);
+
+    void* IGNITE_CALL IgniteCacheWithSkipStore(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteCacheWithNoRetries(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteCacheWithExpiryPolicy(gcj::JniContext* ctx, void* obj, long long create, long long update, long long access);
+    void* IGNITE_CALL IgniteCacheWithAsync(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteCacheWithKeepPortable(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteCacheClear(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteCacheRemoveAll(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteCacheOutOpQueryCursor(gcj::JniContext* ctx, void* obj, int type, long long memPtr);
+    void* IGNITE_CALL IgniteCacheOutOpContinuousQuery(gcj::JniContext* ctx, void* obj, int type, long long memPtr);
+    void* IGNITE_CALL IgniteCacheIterator(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteCacheLocalIterator(gcj::JniContext* ctx, void* obj, int peekModes);
+    void IGNITE_CALL IgniteCacheEnterLock(gcj::JniContext* ctx, void* obj, long long id);
+    void IGNITE_CALL IgniteCacheExitLock(gcj::JniContext* ctx, void* obj, long long id);
+    bool IGNITE_CALL IgniteCacheTryEnterLock(gcj::JniContext* ctx, void* obj, long long id, long long timeout);
+    void IGNITE_CALL IgniteCacheCloseLock(gcj::JniContext* ctx, void* obj, long long id);
+    void IGNITE_CALL IgniteCacheRebalance(gcj::JniContext* ctx, void* obj, long long futId);
+    int IGNITE_CALL IgniteCacheSize(gcj::JniContext* ctx, void* obj, int peekModes, bool loc);
+
+    void IGNITE_CALL IgniteCacheStoreCallbackInvoke(gcj::JniContext* ctx, void* obj, long long memPtr);
+
+    void IGNITE_CALL IgniteComputeWithNoFailover(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteComputeWithTimeout(gcj::JniContext* ctx, void* obj, long long timeout);
+    void IGNITE_CALL IgniteComputeExecuteNative(gcj::JniContext* ctx, void* obj, long long taskPtr, long long topVer);
+
+    void IGNITE_CALL IgniteContinuousQueryClose(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteContinuousQueryGetInitialQueryCursor(gcj::JniContext* ctx, void* obj);
+
+    void IGNITE_CALL IgniteDataStreamerListenTopology(gcj::JniContext* ctx, void* obj, long long ptr);
+    bool IGNITE_CALL IgniteDataStreamerAllowOverwriteGet(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteDataStreamerAllowOverwriteSet(gcj::JniContext* ctx, void* obj, bool val);
+    bool IGNITE_CALL IgniteDataStreamerSkipStoreGet(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteDataStreamerSkipStoreSet(gcj::JniContext* ctx, void* obj, bool val);
+    int IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeGet(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteDataStreamerPerNodeBufferSizeSet(gcj::JniContext* ctx, void* obj, int val);
+    int IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsGet(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteDataStreamerPerNodeParallelOperationsSet(gcj::JniContext* ctx, void* obj, int val);
+
+    void* IGNITE_CALL IgniteMessagingWithAsync(gcj::JniContext* ctx, void* obj);
+
+    void* IGNITE_CALL IgniteProjectionForOthers(gcj::JniContext* ctx, void* obj, void* prj);
+    void* IGNITE_CALL IgniteProjectionForRemotes(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProjectionForDaemons(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProjectionForRandom(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProjectionForOldest(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProjectionForYoungest(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteProjectionResetMetrics(gcj::JniContext* ctx, void* obj);
+    void* IGNITE_CALL IgniteProjectionOutOpRet(gcj::JniContext* ctx, void* obj, int type, long long memPtr);
+
+    void IGNITE_CALL IgniteQueryCursorIterator(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteQueryCursorClose(gcj::JniContext* ctx, void* obj);
+
+    long long IGNITE_CALL IgniteTransactionsStart(gcj::JniContext* ctx, void* obj, int concurrency, int isolation, long long timeout, int txSize);
+    int IGNITE_CALL IgniteTransactionsCommit(gcj::JniContext* ctx, void* obj, long long id);
+    void IGNITE_CALL IgniteTransactionsCommitAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId);
+    int IGNITE_CALL IgniteTransactionsRollback(gcj::JniContext* ctx, void* obj, long long id);
+    void IGNITE_CALL IgniteTransactionsRollbackAsync(gcj::JniContext* ctx, void* obj, long long id, long long futId);
+    int IGNITE_CALL IgniteTransactionsClose(gcj::JniContext* ctx, void* obj, long long id);
+    int IGNITE_CALL IgniteTransactionsState(gcj::JniContext* ctx, void* obj, long long id);
+    bool IGNITE_CALL IgniteTransactionsSetRollbackOnly(gcj::JniContext* ctx, void* obj, long long id);
+    void IGNITE_CALL IgniteTransactionsResetMetrics(gcj::JniContext* ctx, void* obj);
+
+    void* IGNITE_CALL IgniteAcquire(gcj::JniContext* ctx, void* obj);
+    void IGNITE_CALL IgniteRelease(void* obj);
+
+    void IGNITE_CALL IgniteThrowToJava(gcj::JniContext* ctx, char* errMsg);
+    
+    int IGNITE_CALL IgniteHandlersSize();
+
+    void* IGNITE_CALL IgniteCreateContext(char** opts, int optsLen, gcj::JniHandlers* cbs);
+    void IGNITE_CALL IgniteDeleteContext(gcj::JniContext* ctx);
+
+    void IGNITE_CALL IgniteDestroyJvm(gcj::JniContext* ctx);
+
+    void* IGNITE_CALL IgniteEventsWithAsync(gcj::JniContext* ctx, void* obj);
+    bool IGNITE_CALL IgniteEventsStopLocalListen(gcj::JniContext* ctx, void* obj, long long hnd);
+    void IGNITE_CALL IgniteEventsLocalListen(gcj::JniContext* ctx, void* obj, long long hnd, int type);
+    bool IGNITE_CALL IgniteEventsIsEnabled(gcj::JniContext* ctx, void* obj, int type);
+        
+	void* IGNITE_CALL IgniteServicesWithAsync(gcj::JniContext* ctx, void* obj);
+	void* IGNITE_CALL IgniteServicesWithServerKeepPortable(gcj::JniContext* ctx, void* obj);
+	void IGNITE_CALL IgniteServicesCancel(gcj::JniContext* ctx, void* obj, char* name);
+	void IGNITE_CALL IgniteServicesCancelAll(gcj::JniContext* ctx, void* obj);
+	void* IGNITE_CALL IgniteServicesGetServiceProxy(gcj::JniContext* ctx, void* obj, char* name, bool sticky);
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/include/ignite/common/java.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/include/ignite/common/java.h b/modules/platform/src/main/cpp/common/include/ignite/common/java.h
new file mode 100644
index 0000000..426ac68
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/include/ignite/common/java.h
@@ -0,0 +1,652 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_COMMON_JVM
+#define _IGNITE_COMMON_JVM
+
+#include <jni.h>
+
+#include "ignite/common/common.h"
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace java
+        {
+            /* Error constants. */
+            const int IGNITE_JNI_ERR_SUCCESS = 0;
+            const int IGNITE_JNI_ERR_GENERIC = 1;
+            const int IGNITE_JNI_ERR_JVM_INIT = 2;
+            const int IGNITE_JNI_ERR_JVM_ATTACH = 3;
+
+            /* Handlers for callbacks from Java. */
+            typedef long long(JNICALL *CacheStoreCreateHandler)(void* target, long long memPtr);
+            typedef int(JNICALL *CacheStoreInvokeHandler)(void* target, long long objPtr, long long memPtr, void* cb);
+            typedef void(JNICALL *CacheStoreDestroyHandler)(void* target, long long objPtr);
+            typedef long long(JNICALL *CacheStoreSessionCreateHandler)(void* target, long long storePtr);
+
+            typedef long long(JNICALL *CacheEntryFilterCreateHandler)(void* target, long long memPtr);
+            typedef int(JNICALL *CacheEntryFilterApplyHandler)(void* target, long long ptr, long long memPtr);
+            typedef void(JNICALL *CacheEntryFilterDestroyHandler)(void* target, long long ptr);
+
+            typedef void(JNICALL *CacheInvokeHandler)(void* target, long long inMemPtr, long long outMemPtr);
+
+            typedef void(JNICALL *ComputeTaskMapHandler)(void* target, long long taskPtr, long long inMemPtr, long long outMemPtr);
+            typedef int(JNICALL *ComputeTaskJobResultHandler)(void* target, long long taskPtr, long long jobPtr, long long memPtr);
+            typedef void(JNICALL *ComputeTaskReduceHandler)(void* target, long long taskPtr);
+            typedef void(JNICALL *ComputeTaskCompleteHandler)(void* target, long long taskPtr, long long memPtr);
+            typedef int(JNICALL *ComputeJobSerializeHandler)(void* target, long long jobPtr, long long memPtr);
+            typedef long long(JNICALL *ComputeJobCreateHandler)(void* target, long long memPtr);
+            typedef void(JNICALL *ComputeJobExecuteHandler)(void* target, long long jobPtr, int cancel, long long memPtr);
+            typedef void(JNICALL *ComputeJobCancelHandler)(void* target, long long jobPtr);
+            typedef void(JNICALL *ComputeJobDestroyHandler)(void* target, long long jobPtr);
+
+            typedef void(JNICALL *ContinuousQueryListenerApplyHandler)(void* target, long long lsnrPtr, long long memPtr);
+            typedef long long(JNICALL *ContinuousQueryFilterCreateHandler)(void* target, long long memPtr);
+            typedef int(JNICALL *ContinuousQueryFilterApplyHandler)(void* target, long long filterPtr, long long memPtr);
+            typedef void(JNICALL *ContinuousQueryFilterReleaseHandler)(void* target, long long filterPtr);
+
+			typedef void(JNICALL *DataStreamerTopologyUpdateHandler)(void* target, long long ldrPtr, long long topVer, int topSize);
+			typedef void(JNICALL *DataStreamerStreamReceiverInvokeHandler)(void* target, long long ptr, void* cache, long long memPtr, unsigned char keepPortable);
+
+            typedef void(JNICALL *FutureByteResultHandler)(void* target, long long futAddr, int res);
+            typedef void(JNICALL *FutureBoolResultHandler)(void* target, long long futAddr, int res);
+            typedef void(JNICALL *FutureShortResultHandler)(void* target, long long futAddr, int res);
+            typedef void(JNICALL *FutureCharResultHandler)(void* target, long long futAddr, int res);
+            typedef void(JNICALL *FutureIntResultHandler)(void* target, long long futAddr, int res);
+            typedef void(JNICALL *FutureFloatResultHandler)(void* target, long long futAddr, float res);
+            typedef void(JNICALL *FutureLongResultHandler)(void* target, long long futAddr, long long res);
+            typedef void(JNICALL *FutureDoubleResultHandler)(void* target, long long futAddr, double res);
+            typedef void(JNICALL *FutureObjectResultHandler)(void* target, long long futAddr, long long memPtr);
+            typedef void(JNICALL *FutureNullResultHandler)(void* target, long long futAddr);
+            typedef void(JNICALL *FutureErrorHandler)(void* target, long long futAddr, long long memPtr);
+
+            typedef void(JNICALL *LifecycleEventHandler)(void* target, long long ptr, int evt);
+
+            typedef void(JNICALL *MemoryReallocateHandler)(void* target, long long memPtr, int cap);
+
+            typedef long long(JNICALL *MessagingFilterCreateHandler)(void* target, long long memPtr);
+            typedef int(JNICALL *MessagingFilterApplyHandler)(void* target, long long ptr, long long memPtr);
+            typedef void(JNICALL *MessagingFilterDestroyHandler)(void* target, long long ptr);
+
+            typedef long long(JNICALL *EventFilterCreateHandler)(void* target, long long memPtr);
+            typedef int(JNICALL *EventFilterApplyHandler)(void* target, long long ptr, long long memPtr);
+            typedef void(JNICALL *EventFilterDestroyHandler)(void* target, long long ptr);
+
+			typedef long long(JNICALL *ServiceInitHandler)(void* target, long long memPtr);
+			typedef void(JNICALL *ServiceExecuteHandler)(void* target, long long svcPtr, long long memPtr);
+			typedef void(JNICALL *ServiceCancelHandler)(void* target, long long svcPtr, long long memPtr);
+			typedef void(JNICALL *ServiceInvokeMethodHandler)(void* target, long long svcPtr, long long inMemPtr, long long outMemPtr);
+			typedef int(JNICALL *ClusterNodeFilterApplyHandler)(void* target, long long memPtr);
+
+            typedef long long(JNICALL *NodeInfoHandler)(void* target, long long memPtr);
+
+            typedef void(JNICALL *OnStartHandler)(void* target, long long memPtr);
+            typedef void(JNICALL *OnStopHandler)(void* target);
+            typedef void(JNICALL *ErrorHandler)(void* target, int errCode, const char* errClsChars, int errClsCharsLen, const char* errMsgChars, int errMsgCharsLen, void* errData, int errDataLen);
+
+            typedef long long(JNICALL *ExtensionCallbackInLongOutLongHandler)(void* target, int typ, long long arg1);
+            typedef long long(JNICALL *ExtensionCallbackInLongLongOutLongHandler)(void* target, int typ, long long arg1, long long arg2);
+
+            /**
+             * JNI handlers holder.
+             */
+            struct JniHandlers {
+                void* target;
+
+                CacheStoreCreateHandler cacheStoreCreate;
+                CacheStoreInvokeHandler cacheStoreInvoke;
+                CacheStoreDestroyHandler cacheStoreDestroy;
+                CacheStoreSessionCreateHandler cacheStoreSessionCreate;
+
+                CacheEntryFilterCreateHandler cacheEntryFilterCreate;
+                CacheEntryFilterApplyHandler cacheEntryFilterApply;
+                CacheEntryFilterDestroyHandler cacheEntryFilterDestroy;
+
+                CacheInvokeHandler cacheInvoke;
+
+                ComputeTaskMapHandler computeTaskMap;
+                ComputeTaskJobResultHandler computeTaskJobRes;
+                ComputeTaskReduceHandler computeTaskReduce;
+                ComputeTaskCompleteHandler computeTaskComplete;
+                ComputeJobSerializeHandler computeJobSerialize;
+                ComputeJobCreateHandler computeJobCreate;
+                ComputeJobExecuteHandler computeJobExec;
+                ComputeJobCancelHandler computeJobCancel;
+                ComputeJobDestroyHandler computeJobDestroy;
+
+                ContinuousQueryListenerApplyHandler contQryLsnrApply;
+                ContinuousQueryFilterCreateHandler contQryFilterCreate;
+                ContinuousQueryFilterApplyHandler contQryFilterApply;
+                ContinuousQueryFilterReleaseHandler contQryFilterRelease;
+
+				DataStreamerTopologyUpdateHandler dataStreamerTopologyUpdate;
+				DataStreamerStreamReceiverInvokeHandler streamReceiverInvoke;
+
+                FutureByteResultHandler futByteRes;
+                FutureBoolResultHandler futBoolRes;
+                FutureShortResultHandler futShortRes;
+                FutureCharResultHandler futCharRes;
+                FutureIntResultHandler futIntRes;
+                FutureFloatResultHandler futFloatRes;
+                FutureLongResultHandler futLongRes;
+                FutureDoubleResultHandler futDoubleRes;
+                FutureObjectResultHandler futObjRes;
+                FutureNullResultHandler futNullRes;
+                FutureErrorHandler futErr;
+
+                LifecycleEventHandler lifecycleEvt;
+
+                MemoryReallocateHandler memRealloc;
+
+                MessagingFilterCreateHandler messagingFilterCreate;
+                MessagingFilterApplyHandler messagingFilterApply;
+                MessagingFilterDestroyHandler messagingFilterDestroy;
+                
+                EventFilterCreateHandler eventFilterCreate;
+                EventFilterApplyHandler eventFilterApply;
+                EventFilterDestroyHandler eventFilterDestroy;
+
+				ServiceInitHandler serviceInit;
+				ServiceExecuteHandler serviceExecute;
+				ServiceCancelHandler serviceCancel;
+				ServiceInvokeMethodHandler serviceInvokeMethod;
+				
+				ClusterNodeFilterApplyHandler clusterNodeFilterApply;
+
+                NodeInfoHandler nodeInfo;
+
+                OnStartHandler onStart;
+                OnStopHandler onStop;
+                ErrorHandler error;
+
+                ExtensionCallbackInLongOutLongHandler extensionCallbackInLongOutLong;
+                ExtensionCallbackInLongLongOutLongHandler extensionCallbackInLongLongOutLong;
+            };
+
+            /**
+             * JNI Java members.
+             */
+            struct JniJavaMembers {
+                jclass c_Class;
+                jmethodID m_Class_getName;
+
+                jclass c_Throwable;
+                jmethodID m_Throwable_getMessage;
+                jmethodID m_Throwable_printStackTrace;
+
+                /**
+                 * Constructor.
+                 */
+                void Initialize(JNIEnv* env);
+
+                /**
+                 * Destroy members releasing all allocated classes.
+                 */
+                void Destroy(JNIEnv* env);
+
+                /**
+                 * Write error information.
+                 */
+                bool WriteErrorInfo(JNIEnv* env, char** errClsName, int* errClsNameLen, char** errMsg, int* errMsgLen);
+            };
+
+            /**
+             * JNI members.
+             */
+            struct JniMembers {
+                jclass c_PlatformAbstractQryCursor;
+                jmethodID m_PlatformAbstractQryCursor_iter;
+                jmethodID m_PlatformAbstractQryCursor_iterHasNext;
+                jmethodID m_PlatformAbstractQryCursor_close;
+
+                jclass c_PlatformAffinity;
+                jmethodID m_PlatformAffinity_partitions;
+
+                jclass c_PlatformCache;
+                jmethodID m_PlatformCache_withSkipStore;
+                jmethodID m_PlatformCache_withNoRetries;
+                jmethodID m_PlatformCache_withExpiryPolicy;
+                jmethodID m_PlatformCache_withAsync;
+                jmethodID m_PlatformCache_withKeepPortable;
+                jmethodID m_PlatformCache_clear;
+                jmethodID m_PlatformCache_removeAll;
+                jmethodID m_PlatformCache_iterator;
+                jmethodID m_PlatformCache_localIterator;
+                jmethodID m_PlatformCache_enterLock;
+                jmethodID m_PlatformCache_exitLock;
+                jmethodID m_PlatformCache_tryEnterLock;
+                jmethodID m_PlatformCache_closeLock;
+                jmethodID m_PlatformCache_rebalance;
+                jmethodID m_PlatformCache_size;
+
+                jclass c_PlatformCacheStoreCallback;
+                jmethodID m_PlatformCacheStoreCallback_invoke;
+
+                jclass c_IgniteException;
+
+                jclass c_PlatformClusterGroup;
+                jmethodID m_PlatformClusterGroup_forOthers;
+                jmethodID m_PlatformClusterGroup_forRemotes;
+                jmethodID m_PlatformClusterGroup_forDaemons;
+                jmethodID m_PlatformClusterGroup_forRandom;
+                jmethodID m_PlatformClusterGroup_forOldest;
+                jmethodID m_PlatformClusterGroup_forYoungest;
+                jmethodID m_PlatformClusterGroup_resetMetrics;
+
+                jclass c_PlatformCompute;
+                jmethodID m_PlatformCompute_withNoFailover;
+                jmethodID m_PlatformCompute_withTimeout;
+                jmethodID m_PlatformCompute_executeNative;
+
+                jclass c_PlatformContinuousQuery;
+                jmethodID m_PlatformContinuousQuery_close;
+                jmethodID m_PlatformContinuousQuery_getInitialQueryCursor;
+
+                jclass c_PlatformDataStreamer;
+                jmethodID m_PlatformDataStreamer_listenTopology;
+                jmethodID m_PlatformDataStreamer_getAllowOverwrite;
+                jmethodID m_PlatformDataStreamer_setAllowOverwrite;
+                jmethodID m_PlatformDataStreamer_getSkipStore;
+                jmethodID m_PlatformDataStreamer_setSkipStore;
+                jmethodID m_PlatformDataStreamer_getPerNodeBufSize;
+                jmethodID m_PlatformDataStreamer_setPerNodeBufSize;
+                jmethodID m_PlatformDataStreamer_getPerNodeParallelOps;
+                jmethodID m_PlatformDataStreamer_setPerNodeParallelOps;
+                
+                jclass c_PlatformEvents;
+                jmethodID m_PlatformEvents_withAsync;
+                jmethodID m_PlatformEvents_stopLocalListen;
+                jmethodID m_PlatformEvents_localListen;
+                jmethodID m_PlatformEvents_isEnabled;
+                
+				jclass c_PlatformServices;
+				jmethodID m_PlatformServices_withAsync;
+				jmethodID m_PlatformServices_withServerKeepPortable;
+				jmethodID m_PlatformServices_cancel;
+				jmethodID m_PlatformServices_cancelAll;
+				jmethodID m_PlatformServices_serviceProxy;
+
+				jclass c_PlatformIgnition;
+                jmethodID m_PlatformIgnition_start;
+                jmethodID m_PlatformIgnition_instance;
+                jmethodID m_PlatformIgnition_environmentPointer;
+                jmethodID m_PlatformIgnition_stop;
+                jmethodID m_PlatformIgnition_stopAll;
+
+                jclass c_PlatformMessaging;
+                jmethodID m_PlatformMessaging_withAsync;
+
+                jclass c_PlatformProcessor;
+                jmethodID m_PlatformProcessor_releaseStart;
+                jmethodID m_PlatformProcessor_cache;
+                jmethodID m_PlatformProcessor_createCache;
+                jmethodID m_PlatformProcessor_getOrCreateCache;
+                jmethodID m_PlatformProcessor_affinity;
+                jmethodID m_PlatformProcessor_dataStreamer;
+                jmethodID m_PlatformProcessor_transactions;
+                jmethodID m_PlatformProcessor_projection;
+                jmethodID m_PlatformProcessor_compute;
+                jmethodID m_PlatformProcessor_message;
+                jmethodID m_PlatformProcessor_events;
+                jmethodID m_PlatformProcessor_services;
+                jmethodID m_PlatformProcessor_extensions;
+
+                jclass c_PlatformTarget;
+                jmethodID m_PlatformTarget_inStreamOutLong;
+                jmethodID m_PlatformTarget_inStreamOutObject;
+                jmethodID m_PlatformTarget_outLong;
+                jmethodID m_PlatformTarget_outStream;
+                jmethodID m_PlatformTarget_outObject;
+                jmethodID m_PlatformTarget_inStreamOutStream;
+                jmethodID m_PlatformTarget_inObjectStreamOutStream;
+                jmethodID m_PlatformTarget_listenFuture;
+                jmethodID m_PlatformTarget_listenFutureForOperation;
+
+                jclass c_PlatformTransactions;
+                jmethodID m_PlatformTransactions_txStart;
+                jmethodID m_PlatformTransactions_txCommit;
+                jmethodID m_PlatformTransactions_txCommitAsync;
+                jmethodID m_PlatformTransactions_txRollback;
+                jmethodID m_PlatformTransactions_txRollbackAsync;
+                jmethodID m_PlatformTransactions_txState;
+                jmethodID m_PlatformTransactions_txSetRollbackOnly;
+                jmethodID m_PlatformTransactions_txClose;
+                jmethodID m_PlatformTransactions_resetMetrics;
+
+                jclass c_PlatformUtils;
+                jmethodID m_PlatformUtils_reallocate;
+                jmethodID m_PlatformUtils_errData;
+
+                /**
+                 * Constructor.
+                 */
+                void Initialize(JNIEnv* env);
+
+                /**
+                 * Destroy members releasing all allocated classes.
+                 */
+                void Destroy(JNIEnv* env);
+            };
+
+            /**
+             * JNI JVM wrapper.
+             */
+            class IGNITE_IMPORT_EXPORT JniJvm {
+            public:
+                /**
+                 * Default constructor for uninitialized JVM.
+                 */
+                JniJvm();
+
+                /**
+                 * Constructor.
+                 *
+                 * @param jvm JVM.
+                 * @param javaMembers Java members.
+                 * @param members Members.
+                 */
+                JniJvm(JavaVM* jvm, JniJavaMembers javaMembers, JniMembers members);
+
+                /**
+                 * Get JVM.
+                 *
+                 * @param JVM.
+                 */
+                JavaVM* GetJvm();
+
+                /**
+                 * Get Java members.
+                 *
+                 * @param Java members.
+                 */
+                JniJavaMembers& GetJavaMembers();
+
+                /**
+                 * Get members.
+                 *
+                 * @param Members.
+                 */
+                JniMembers& GetMembers();
+            private:
+                /** JVM. */
+                JavaVM* jvm;
+
+                /** Java members. */
+                JniJavaMembers javaMembers;
+
+                /** Members. */
+                JniMembers members;
+            };
+
+            /**
+             * JNI error information.
+             */
+            struct IGNITE_IMPORT_EXPORT JniErrorInfo
+            {
+                int code;
+                char* errCls;
+                char* errMsg;
+
+                /**
+                 * Default constructor. Creates empty error info.
+                 */
+                JniErrorInfo();
+
+                /**
+                 * Constructor.
+                 *
+                 * @param code Code.
+                 * @param errCls Error class.
+                 * @param errMsg Error message.
+                 */
+                JniErrorInfo(int code, const char* errCls, const char* errMsg);
+
+                /**
+                 * Copy constructor.
+                 *
+                 * @param other Other instance.
+                 */
+                JniErrorInfo(const JniErrorInfo& other);
+
+                /**
+                 * Assignment operator overload.
+                 *
+                 * @param other Other instance.
+                 * @return This instance.
+                 */
+                JniErrorInfo& operator=(const JniErrorInfo& other);
+
+                /**
+                 * Destructor.
+                 */
+                ~JniErrorInfo();
+            };
+
+            /**
+             * Unmanaged context.
+             */
+            class IGNITE_IMPORT_EXPORT JniContext {
+            public:
+                static JniContext* Create(char** opts, int optsLen, JniHandlers hnds);
+                static JniContext* Create(char** opts, int optsLen, JniHandlers hnds, JniErrorInfo* errInfo);
+                static int Reallocate(long long memPtr, int cap);
+                static void Detach();
+                static void Release(jobject obj);
+
+                jobject IgnitionStart(char* cfgPath, char* gridName, int factoryId, long long dataPtr);
+                jobject IgnitionStart(char* cfgPath, char* gridName, int factoryId, long long dataPtr, JniErrorInfo* errInfo);
+                jobject IgnitionInstance(char* gridName);
+                jobject IgnitionInstance(char* gridName, JniErrorInfo* errInfo);
+                long long IgnitionEnvironmentPointer(char* gridName);
+                long long IgnitionEnvironmentPointer(char* gridName, JniErrorInfo* errInfo);
+                bool IgnitionStop(char* gridName, bool cancel);
+                bool IgnitionStop(char* gridName, bool cancel, JniErrorInfo* errInfo);
+                void IgnitionStopAll(bool cancel);
+                void IgnitionStopAll(bool cancel, JniErrorInfo* errInfo);
+                
+                void ProcessorReleaseStart(jobject obj);
+                jobject ProcessorProjection(jobject obj);
+                jobject ProcessorCache(jobject obj, const char* name);
+                jobject ProcessorCache(jobject obj, const char* name, JniErrorInfo* errInfo);
+                jobject ProcessorCreateCache(jobject obj, const char* name);
+                jobject ProcessorCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
+                jobject ProcessorGetOrCreateCache(jobject obj, const char* name);
+                jobject ProcessorGetOrCreateCache(jobject obj, const char* name, JniErrorInfo* errInfo);
+                jobject ProcessorAffinity(jobject obj, const char* name);
+                jobject ProcessorDataStreamer(jobject obj, const char* name, bool keepPortable);
+                jobject ProcessorTransactions(jobject obj);
+                jobject ProcessorCompute(jobject obj, jobject prj);
+                jobject ProcessorMessage(jobject obj, jobject prj);
+                jobject ProcessorEvents(jobject obj, jobject prj);
+                jobject ProcessorServices(jobject obj, jobject prj);
+                jobject ProcessorExtensions(jobject obj);
+                
+                long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
+                void TargetInStreamOutStream(jobject obj, int opType, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL);
+                jobject TargetInStreamOutObject(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
+                void TargetInObjectStreamOutStream(jobject obj, int opType, void* arg, long long inMemPtr, long long outMemPtr, JniErrorInfo* errInfo = NULL);
+                long long TargetOutLong(jobject obj, int opType, JniErrorInfo* errInfo = NULL);
+                void TargetOutStream(jobject obj, int opType, long long memPtr, JniErrorInfo* errInfo = NULL);
+                jobject TargetOutObject(jobject obj, int opType, JniErrorInfo* errInfo = NULL);
+                void TargetListenFuture(jobject obj, long long futId, int typ);
+                void TargetListenFutureForOperation(jobject obj, long long futId, int typ, int opId);
+                
+                int AffinityPartitions(jobject obj);
+
+                jobject CacheWithSkipStore(jobject obj);
+                jobject CacheWithNoRetries(jobject obj);
+                jobject CacheWithExpiryPolicy(jobject obj, long long create, long long update, long long access);
+                jobject CacheWithAsync(jobject obj);
+                jobject CacheWithKeepPortable(jobject obj);
+                void CacheClear(jobject obj, JniErrorInfo* errInfo = NULL);
+                void CacheRemoveAll(jobject obj, JniErrorInfo* errInfo = NULL);
+                jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
+                jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr);
+                jobject CacheIterator(jobject obj);
+                jobject CacheLocalIterator(jobject obj, int peekModes);
+                void CacheEnterLock(jobject obj, long long id);
+                void CacheExitLock(jobject obj, long long id);
+                bool CacheTryEnterLock(jobject obj, long long id, long long timeout);
+                void CacheCloseLock(jobject obj, long long id);
+                void CacheRebalance(jobject obj, long long futId);
+                int CacheSize(jobject obj, int peekModes, bool loc, JniErrorInfo* errInfo = NULL);
+
+                void CacheStoreCallbackInvoke(jobject obj, long long memPtr);
+
+                void ComputeWithNoFailover(jobject obj);
+                void ComputeWithTimeout(jobject obj, long long timeout);
+                void ComputeExecuteNative(jobject obj, long long taskPtr, long long topVer);
+
+                void ContinuousQueryClose(jobject obj);
+                void* ContinuousQueryGetInitialQueryCursor(jobject obj);
+
+                void DataStreamerListenTopology(jobject obj, long long ptr);
+                bool DataStreamerAllowOverwriteGet(jobject obj);
+                void DataStreamerAllowOverwriteSet(jobject obj, bool val);
+                bool DataStreamerSkipStoreGet(jobject obj);
+                void DataStreamerSkipStoreSet(jobject obj, bool val);
+                int DataStreamerPerNodeBufferSizeGet(jobject obj);
+                void DataStreamerPerNodeBufferSizeSet(jobject obj, int val);
+                int DataStreamerPerNodeParallelOperationsGet(jobject obj);
+                void DataStreamerPerNodeParallelOperationsSet(jobject obj, int val);
+
+                jobject MessagingWithAsync(jobject obj);
+
+                jobject ProjectionForOthers(jobject obj, jobject prj);
+                jobject ProjectionForRemotes(jobject obj);
+                jobject ProjectionForDaemons(jobject obj);
+                jobject ProjectionForRandom(jobject obj);
+                jobject ProjectionForOldest(jobject obj);
+                jobject ProjectionForYoungest(jobject obj);
+                void ProjectionResetMetrics(jobject obj);
+                jobject ProjectionOutOpRet(jobject obj, int type, long long memPtr);
+
+                void QueryCursorIterator(jobject obj, JniErrorInfo* errInfo = NULL);
+                bool QueryCursorIteratorHasNext(jobject obj, JniErrorInfo* errInfo = NULL);
+                void QueryCursorClose(jobject obj, JniErrorInfo* errInfo = NULL);
+
+                long long TransactionsStart(jobject obj, int concurrency, int isolation, long long timeout, int txSize);
+                int TransactionsCommit(jobject obj, long long id);
+                void TransactionsCommitAsync(jobject obj, long long id, long long futId);
+                int TransactionsRollback(jobject obj, long long id);
+                void TransactionsRollbackAsync(jobject obj, long long id, long long futId);
+                int TransactionsClose(jobject obj, long long id);
+                int TransactionsState(jobject obj, long long id);
+                bool TransactionsSetRollbackOnly(jobject obj, long long id);
+                void TransactionsResetMetrics(jobject obj);
+
+                jobject EventsWithAsync(jobject obj);
+                bool EventsStopLocalListen(jobject obj, long long hnd);
+                void EventsLocalListen(jobject obj, long long hnd, int type);
+                bool EventsIsEnabled(jobject obj, int type);
+                
+				jobject ServicesWithAsync(jobject obj);
+                jobject ServicesWithServerKeepPortable(jobject obj);
+				void ServicesCancel(jobject obj, char* name);
+				void ServicesCancelAll(jobject obj);
+				void* ServicesGetServiceProxy(jobject obj, char* name, bool sticky);
+
+                jobject Acquire(jobject obj);
+
+                void DestroyJvm();
+                void ThrowToJava(char* errMsg);
+            private:
+                JniJvm* jvm;
+                JniHandlers hnds;
+
+                JniContext(JniJvm* jvm, JniHandlers hnds);
+
+                JNIEnv* Attach();
+                void ExceptionCheck(JNIEnv* env);
+                void ExceptionCheck(JNIEnv* env, JniErrorInfo* errInfo);
+                jobject LocalToGlobal(JNIEnv* env, jobject obj);
+                jobject ProcessorCache0(jobject proc, const char* name, jmethodID mthd, JniErrorInfo* errInfo);
+            };
+
+            JNIEXPORT jlong JNICALL JniCacheStoreCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT jint JNICALL JniCacheStoreInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr, jobject cb);
+            JNIEXPORT void JNICALL JniCacheStoreDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr);
+            JNIEXPORT jlong JNICALL JniCacheStoreSessionCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong storePtr);
+
+            JNIEXPORT jlong JNICALL JniCacheEntryFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT jint JNICALL JniCacheEntryFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr, jlong memPtr);
+            JNIEXPORT void JNICALL JniCacheEntryFilterDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong objPtr);
+
+            JNIEXPORT void JNICALL JniCacheInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong inMemPtr, jlong outMemPtr);
+
+            JNIEXPORT void JNICALL JniComputeTaskMap(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr, jlong inMemPtr, jlong outMemPtr);
+            JNIEXPORT jint JNICALL JniComputeTaskJobResult(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr, jlong jobPtr, jlong memPtr);
+            JNIEXPORT void JNICALL JniComputeTaskReduce(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr);
+            JNIEXPORT void JNICALL JniComputeTaskComplete(JNIEnv *env, jclass cls, jlong envPtr, jlong taskPtr, jlong memPtr);
+            JNIEXPORT jint JNICALL JniComputeJobSerialize(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr, jlong memPtr);
+            JNIEXPORT jlong JNICALL JniComputeJobCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT void JNICALL JniComputeJobExecute(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr, jint cancel, jlong memPtr);
+            JNIEXPORT void JNICALL JniComputeJobCancel(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr);
+            JNIEXPORT void JNICALL JniComputeJobDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong jobPtr);
+
+            JNIEXPORT void JNICALL JniContinuousQueryListenerApply(JNIEnv *env, jclass cls, jlong envPtr, jlong cbPtr, jlong memPtr);
+            JNIEXPORT jlong JNICALL JniContinuousQueryFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT jint JNICALL JniContinuousQueryFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong filterPtr, jlong memPtr);
+            JNIEXPORT void JNICALL JniContinuousQueryFilterRelease(JNIEnv *env, jclass cls, jlong envPtr, jlong filterPtr);
+
+			JNIEXPORT void JNICALL JniDataStreamerTopologyUpdate(JNIEnv *env, jclass cls, jlong envPtr, jlong ldrPtr, jlong topVer, jint topSize);
+			JNIEXPORT void JNICALL JniDataStreamerStreamReceiverInvoke(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jobject cache, jlong memPtr, jboolean keepPortable);
+
+            JNIEXPORT void JNICALL JniFutureByteResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res);
+            JNIEXPORT void JNICALL JniFutureBoolResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res);
+            JNIEXPORT void JNICALL JniFutureShortResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res);
+            JNIEXPORT void JNICALL JniFutureCharResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res);
+            JNIEXPORT void JNICALL JniFutureIntResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jint res);
+            JNIEXPORT void JNICALL JniFutureFloatResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jfloat res);
+            JNIEXPORT void JNICALL JniFutureLongResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jlong res);
+            JNIEXPORT void JNICALL JniFutureDoubleResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jdouble res);
+            JNIEXPORT void JNICALL JniFutureObjectResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jlong memPtr);
+            JNIEXPORT void JNICALL JniFutureNullResult(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr);
+            JNIEXPORT void JNICALL JniFutureError(JNIEnv *env, jclass cls, jlong envPtr, jlong futPtr, jlong memPtr);
+
+            JNIEXPORT void JNICALL JniLifecycleEvent(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jint evt);
+
+            JNIEXPORT void JNICALL JniMemoryReallocate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr, jint cap);
+
+            JNIEXPORT jlong JNICALL JniMessagingFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT jint JNICALL JniMessagingFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jlong memPtr);
+            JNIEXPORT void JNICALL JniMessagingFilterDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr);
+            
+            JNIEXPORT jlong JNICALL JniEventFilterCreate(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT jint JNICALL JniEventFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr, jlong memPtr);
+            JNIEXPORT void JNICALL JniEventFilterDestroy(JNIEnv *env, jclass cls, jlong envPtr, jlong ptr);
+
+			JNIEXPORT jlong JNICALL JniServiceInit(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+			JNIEXPORT void JNICALL JniServiceExecute(JNIEnv *env, jclass cls, jlong envPtr, jlong svcPtr, jlong memPtr);
+			JNIEXPORT void JNICALL JniServiceCancel(JNIEnv *env, jclass cls, jlong envPtr, jlong svcPtr, jlong memPtr);
+			JNIEXPORT void JNICALL JniServiceInvokeMethod(JNIEnv *env, jclass cls, jlong envPtr, jlong svcPtr, jlong inMemPtr, jlong outMemPtr);
+			JNIEXPORT jint JNICALL JniClusterNodeFilterApply(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+
+            JNIEXPORT jlong JNICALL JniNodeInfo(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+
+            JNIEXPORT void JNICALL JniOnStart(JNIEnv *env, jclass cls, jlong envPtr, jlong memPtr);
+            JNIEXPORT void JNICALL JniOnStop(JNIEnv *env, jclass cls, jlong envPtr);
+
+            JNIEXPORT jlong JNICALL JniExtensionCallbackInLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1);
+            JNIEXPORT jlong JNICALL JniExtensionCallbackInLongLongOutLong(JNIEnv *env, jclass cls, jlong envPtr, jint typ, jlong arg1, jlong arg2);
+        }
+    }
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/linux/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/linux/include/Makefile.am b/modules/platform/src/main/cpp/common/os/linux/include/Makefile.am
new file mode 100644
index 0000000..68e45e6
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/linux/include/Makefile.am
@@ -0,0 +1,21 @@
+##
+## 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.
+##
+
+ACLOCAL_AMFLAGS = "-Im4"
+
+nobase_include_HEADERS = ignite/common/common.h \
+                         ignite/common/concurrent_os.h

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/common.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/common.h b/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/common.h
new file mode 100644
index 0000000..6577ad8
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/common.h
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_COMMON_OS
+#define _IGNITE_COMMON_OS
+
+#ifndef __has_attribute
+  #define __has_attribute(x) 0
+#endif
+
+#if (defined(__GNUC__) && ((__GNUC__ > 4) || (__GNUC__ == 4) && (__GNUC_MINOR__ > 2))) || __has_attribute(visibility)
+  #define IGNITE_EXPORT __attribute__((visibility("default")))
+  #define IGNITE_IMPORT __attribute__((visibility("default")))
+#else
+  #define IGNITE_EXPORT
+  #define IGNITE_IMPORT
+#endif
+
+#define IGNITE_CALL
+
+#ifdef IGNITE_IMPL
+    #define IGNITE_IMPORT_EXPORT IGNITE_EXPORT
+#else
+    #define IGNITE_IMPORT_EXPORT IGNITE_IMPORT
+#endif
+
+/**
+ * Common construction to disable copy constructor and assignment for class.
+ */
+#define IGNITE_NO_COPY_ASSIGNMENT(cls) \
+    cls(const cls& src); \
+    cls& operator= (const cls& other);
+
+namespace ignite
+{
+    namespace common
+    {
+        /**
+         * Helper class to manage attached threads.
+         */
+        class AttachHelper 
+        {
+        public:            
+            /**
+             * Destructor.
+             */
+            ~AttachHelper();
+            
+            /**
+             * Callback invoked on successful thread attach ot JVM.
+             */
+            static void OnThreadAttach();
+        private:
+            /**
+             * Helper method to allocate attach key.
+             */
+            static void AllocateAttachKey();
+
+            /**
+             * Attach key destructor.
+             */
+            static void DestroyAttachKey(void* key);
+        };        
+    }
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/concurrent_os.h
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/concurrent_os.h b/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/concurrent_os.h
new file mode 100644
index 0000000..63798b1
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/linux/include/ignite/common/concurrent_os.h
@@ -0,0 +1,394 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_COMMON_CONCURRENT_OS
+#define _IGNITE_COMMON_CONCURRENT_OS
+
+#include <map>
+#include <stdint.h>
+#include <pthread.h>
+
+#include "ignite/common/common.h"
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace concurrent
+        {
+            /**
+             * Static class to manage memory visibility semantics. 
+             */
+            class IGNITE_IMPORT_EXPORT Memory {
+            public:
+                /**
+                 * Full fence. 
+                 */
+                static void Fence();
+            };
+
+            /**
+             * Critical section.
+             */
+            class IGNITE_IMPORT_EXPORT CriticalSection {
+            public:
+                /**
+                 * Constructor.
+                 */
+                CriticalSection();
+
+                /**
+                 * Destructor. 
+                 */
+                ~CriticalSection();
+
+                /**
+                 * Enter critical section.
+                 */
+                void Enter();
+
+                /**
+                 * Leave critical section.
+                 */
+                void Leave();
+            private:
+                pthread_mutex_t mux;
+                
+                IGNITE_NO_COPY_ASSIGNMENT(CriticalSection)
+            };
+
+            /**
+             * Special latch with count = 1.
+             */
+            class IGNITE_IMPORT_EXPORT SingleLatch
+            {                
+            public:
+                /**
+                 * Constructor.
+                 */
+                SingleLatch();
+
+                /**
+                 * Destructor.
+                 */
+                ~SingleLatch();
+
+                /**
+                 * Perform the countdown.
+                 */
+                void CountDown();
+
+                /**
+                 * Await the countdown.
+                 */
+                void Await();
+            private:
+                /** Mutex. */
+                pthread_mutex_t mux;
+
+                /** Condition. */
+                pthread_cond_t cond;
+
+                /** Ready flag. */
+                bool ready;
+                
+                IGNITE_NO_COPY_ASSIGNMENT(SingleLatch)
+            };
+
+            /**
+             * Primitives for atomic access.
+             */
+            class IGNITE_IMPORT_EXPORT Atomics
+            {
+            public:
+                /**
+                 * Update the 32-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return True if update occurred as a result of this call, false otherwise.
+                 */
+                static bool CompareAndSet32(int32_t* ptr, int32_t expVal, int32_t newVal);
+
+                /**
+                 * Update the 32-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return Value which were observed during CAS attempt.
+                 */
+                static int32_t CompareAndSet32Val(int32_t* ptr, int32_t expVal, int32_t newVal);
+
+                /**
+                 * Increment 32-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after increment.
+                 */
+                static int32_t IncrementAndGet32(int32_t* ptr);
+
+                /**
+                 * Decrement 32-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after decrement.
+                 */
+                static int32_t DecrementAndGet32(int32_t* ptr);
+
+                /**
+                 * Update the 64-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return True if update occurred as a result of this call, false otherwise.
+                 */
+                static bool CompareAndSet64(int64_t* ptr, int64_t expVal, int64_t newVal);
+
+                /**
+                 * Update the 64-bit integer value if it is equal to expected value.
+                 *
+                 * @param ptr Pointer.
+                 * @param expVal Expected value.
+                 * @param newVal New value.
+                 * @return Value which were observed during CAS attempt.
+                 */
+                static int64_t CompareAndSet64Val(int64_t* ptr, int64_t expVal, int64_t newVal);
+
+                /**
+                 * Increment 64-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after increment.
+                 */
+                static int64_t IncrementAndGet64(int64_t* ptr);
+
+                /**
+                 * Decrement 64-bit integer and return new value.
+                 *
+                 * @param ptr Pointer.
+                 * @return Value after decrement.
+                 */
+                static int64_t DecrementAndGet64(int64_t* ptr);
+            };
+
+            /**
+             * Thread-local entry.
+             */
+            class IGNITE_IMPORT_EXPORT ThreadLocalEntry
+            {
+            public:
+                /**
+                 * Virtual destructor to allow for correct typed entries cleanup.
+                 */
+                virtual ~ThreadLocalEntry()
+                {
+                    // No-op.
+                }
+            };
+
+            /**
+             * Typed thread-local entry.
+             */
+            template<typename T>
+            class IGNITE_IMPORT_EXPORT ThreadLocalTypedEntry : public ThreadLocalEntry
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param val Value.
+                 */
+                ThreadLocalTypedEntry(T val) : val(val)
+                {
+                    // No-op.
+                }
+
+                ~ThreadLocalTypedEntry()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get value.
+                 *
+                 * @return Value.
+                 */
+                T Get()
+                {
+                    return val;
+                }
+            private:
+                /** Value. */
+                T val;
+            };
+
+            /**
+             * Thread-local abstraction.
+             */
+            class IGNITE_IMPORT_EXPORT ThreadLocal
+            {
+            public:
+                /**
+                 * Get next available index to be used in thread-local storage.
+                 *
+                 * @return Index.
+                 */
+                static int32_t NextIndex();
+
+                /**
+                 * Get value by index.
+                 *
+                 * @param idx Index.
+                 * @return Value associated with the index or NULL.
+                 */
+                template<typename T>
+                static T Get(int32_t idx)
+                {
+                    void* linuxVal = Get0();
+
+                    if (linuxVal)
+                    {
+                        std::map<int32_t, ThreadLocalEntry*>* map =
+                            static_cast<std::map<int32_t, ThreadLocalEntry*>*>(linuxVal);
+
+                        ThreadLocalTypedEntry<T>* entry = static_cast<ThreadLocalTypedEntry<T>*>((*map)[idx]);
+
+                        if (entry)
+                            return entry->Get();
+                    }
+
+                    return T();
+                }
+
+                /**
+                 * Set value at the given index.
+                 *
+                 * @param idx Index.
+                 * @param val Value to be associated with the index.
+                 */
+                template<typename T>
+                static void Set(int32_t idx, const T& val)
+                {
+                    void* linuxVal = Get0();
+
+                    if (linuxVal)
+                    {
+                        std::map<int32_t, ThreadLocalEntry*>* map =
+                            static_cast<std::map<int32_t, ThreadLocalEntry*>*>(linuxVal);
+
+                        ThreadLocalEntry* appVal = (*map)[idx];
+
+                        if (appVal)
+                            delete appVal;
+
+                        (*map)[idx] = new ThreadLocalTypedEntry<T>(val);
+                    }
+                    else
+                    {
+                        std::map<int32_t, ThreadLocalEntry*>* map = new std::map<int32_t, ThreadLocalEntry*>();
+
+                        Set0(map);
+
+                        (*map)[idx] = new ThreadLocalTypedEntry<T>(val);
+                    }
+                }
+
+                /**
+                 * Remove value at the given index.
+                 *
+                 * @param idx Index.
+                 */
+                static void Remove(int32_t idx);
+
+                /**
+                 * Internal thread-local map clear routine.
+                 *
+                 * @param mapPtr Pointer to map.
+                 */
+                static void Clear0(void* mapPtr);
+
+            private:
+                /**
+                 * Internal get routine.
+                 *
+                 * @param Associated value.
+                 */
+                static void* Get0();
+
+                /**
+                 * Internal set routine.
+                 *
+                 * @param ptr Pointer.
+                 */
+                static void Set0(void* ptr);
+            };
+
+            /**
+             * Thread-local instance. Simplifies API avoiding direct index allocations.
+             */
+            template<typename T>
+            class IGNITE_IMPORT_EXPORT ThreadLocalInstance
+            {
+            public:
+                /**
+                 * Constructor.
+                 */
+                ThreadLocalInstance() : idx(ThreadLocal::NextIndex())
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get value.
+                 *
+                 * @return Value.
+                 */
+                T Get()
+                {
+                    return ThreadLocal::Get<T>(idx);
+                }
+
+                /**
+                 * Set instance.
+                 *
+                 * @param val Value.
+                 */
+                void Set(const T& val)
+                {
+                    ThreadLocal::Set<T>(idx, val);
+                }
+
+                /**
+                 * Remove instance.
+                 */
+                void Remove()
+                {
+                    ThreadLocal::Remove(idx);
+                }
+
+            private:
+                /** Index. */
+                int32_t idx;
+            };
+        }
+    }
+}
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/linux/src/common.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/linux/src/common.cpp b/modules/platform/src/main/cpp/common/os/linux/src/common.cpp
new file mode 100644
index 0000000..c0cccdc
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/linux/src/common.cpp
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+#include <pthread.h>
+
+#include "ignite/common/common.h"
+#include "ignite/common/java.h"
+
+using namespace ignite::common::java;
+
+namespace ignite
+{
+    namespace common
+    {
+        /** Key indicating that the thread is attached. */
+        static pthread_key_t attachKey;
+
+        /** Helper to ensure that attach key is allocated only once. */
+        static pthread_once_t attachKeyInit = PTHREAD_ONCE_INIT;
+        
+        AttachHelper::~AttachHelper()
+        {
+            JniContext::Detach();
+        }
+        
+        void AttachHelper::OnThreadAttach()
+        {
+            pthread_once(&attachKeyInit, AllocateAttachKey);
+            
+            void* val = pthread_getspecific(attachKey);
+            
+            if (!val)
+                pthread_setspecific(attachKey, new AttachHelper());
+        }
+        
+        void AttachHelper::AllocateAttachKey()
+        {
+            pthread_key_create(&attachKey, DestroyAttachKey);
+        }   
+        
+        void AttachHelper::DestroyAttachKey(void* key)
+        {
+            delete reinterpret_cast<AttachHelper*>(key);
+        }             
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/modules/platform/src/main/cpp/common/os/linux/src/concurrent_os.cpp
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/os/linux/src/concurrent_os.cpp b/modules/platform/src/main/cpp/common/os/linux/src/concurrent_os.cpp
new file mode 100644
index 0000000..44f0b22
--- /dev/null
+++ b/modules/platform/src/main/cpp/common/os/linux/src/concurrent_os.cpp
@@ -0,0 +1,175 @@
+/*
+ * 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.
+ */
+
+#include "ignite/common/concurrent_os.h"
+
+namespace ignite
+{
+    namespace common
+    {
+        namespace concurrent
+        {
+            /** Key indicating that the thread is attached. */
+            static pthread_key_t tlsKey;
+
+            /** Helper to ensure that attach key is allocated only once. */
+            static pthread_once_t tlsKeyInit = PTHREAD_ONCE_INIT;
+            
+            /**
+             * Routine to destroy TLS key.
+             * 
+             * @param key Key.
+             */
+            void DestroyTlsKey(void* key) {
+                ThreadLocal::Clear0(key);
+            }
+            
+            /**
+             * Routine to allocate TLS key.
+             */
+            void AllocateTlsKey() {
+                pthread_key_create(&tlsKey, DestroyTlsKey);
+            }
+            
+            void Memory::Fence() {
+                __asm__ volatile ("" ::: "memory");
+            }
+
+            CriticalSection::CriticalSection() {
+                pthread_mutex_init(&mux, NULL);
+                
+                Memory::Fence();
+            }
+
+            CriticalSection::~CriticalSection() {
+                Memory::Fence();
+                
+                pthread_mutex_destroy(&mux);
+            }
+
+            void CriticalSection::Enter() {
+                Memory::Fence();
+                
+                pthread_mutex_lock(&mux);
+            }
+
+            void CriticalSection::Leave() {
+                Memory::Fence();
+                
+                pthread_mutex_unlock(&mux);
+            }
+
+            SingleLatch::SingleLatch()
+            {
+                pthread_mutex_init(&mux, NULL);
+                pthread_cond_init(&cond, NULL);
+                ready = false;
+                
+                Memory::Fence();
+            }
+
+            SingleLatch::~SingleLatch()
+            {
+                Memory::Fence();
+
+                pthread_cond_destroy(&cond);
+                pthread_mutex_destroy(&mux);
+            }
+
+            void SingleLatch::CountDown()
+            {
+                pthread_mutex_lock(&mux);
+                
+                if (!ready) {
+                    ready = true;
+                    
+                    pthread_cond_broadcast(&cond);
+                }
+                
+                pthread_mutex_unlock(&mux);
+                
+                Memory::Fence();
+            }
+
+            void SingleLatch::Await()
+            {
+                pthread_mutex_lock(&mux);
+                
+                while (!ready)
+                    pthread_cond_wait(&cond, &mux);
+                
+                pthread_mutex_unlock(&mux);
+                
+                Memory::Fence();
+            }
+
+            bool Atomics::CompareAndSet32(int32_t* ptr, int32_t expVal, int32_t newVal)
+            {
+                return __sync_bool_compare_and_swap(ptr, expVal, newVal);
+            }
+
+            int32_t Atomics::CompareAndSet32Val(int32_t* ptr, int32_t expVal, int32_t newVal)
+            {
+                return __sync_val_compare_and_swap(ptr, expVal, newVal);
+            }
+
+            int32_t Atomics::IncrementAndGet32(int32_t* ptr)
+            {
+               return __sync_fetch_and_add(ptr, 1) + 1;
+            }
+
+            int32_t Atomics::DecrementAndGet32(int32_t* ptr)
+            {
+               return __sync_fetch_and_sub(ptr, 1) - 1;
+            }
+
+            bool Atomics::CompareAndSet64(int64_t* ptr, int64_t expVal, int64_t newVal)
+            {
+               return __sync_bool_compare_and_swap(ptr, expVal, newVal);
+            }
+
+            int64_t Atomics::CompareAndSet64Val(int64_t* ptr, int64_t expVal, int64_t newVal)
+            {
+               return __sync_val_compare_and_swap(ptr, expVal, newVal);
+            }
+
+            int64_t Atomics::IncrementAndGet64(int64_t* ptr)
+            {
+               return __sync_fetch_and_add(ptr, 1) + 1;
+            }
+
+            int64_t Atomics::DecrementAndGet64(int64_t* ptr)
+            {
+               return __sync_fetch_and_sub(ptr, 1) - 1;
+            }
+
+            void* ThreadLocal::Get0()
+            {
+                pthread_once(&tlsKeyInit, AllocateTlsKey);
+                                
+                return pthread_getspecific(tlsKey);
+            }
+
+            void ThreadLocal::Set0(void* ptr)
+            {
+                pthread_once(&tlsKeyInit, AllocateTlsKey);
+                
+                pthread_setspecific(tlsKey, ptr);
+            }
+        }
+    }
+}
\ No newline at end of file


[29/29] ignite git commit: Merge branches 'ignite-843' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843

Posted by ak...@apache.org.
Merge branches 'ignite-843' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843


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

Branch: refs/heads/ignite-843
Commit: 5777950187975c799ea66b9d2d75c7afca7731af
Parents: fcf3101 5a948dc
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Sep 2 08:59:48 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 2 08:59:48 2015 +0700

----------------------------------------------------------------------
 idea/ignite_codeStyle.xml                       |   11 +-
 modules/core/pom.xml                            |    6 +-
 .../internal/GridEventConsumeHandler.java       |   15 +-
 .../managers/communication/GridIoManager.java   |   50 +-
 .../GridLifecycleAwareMessageFilter.java        |   38 -
 .../eventstorage/GridEventStorageManager.java   |   16 +-
 .../processors/cache/GridCacheAdapter.java      |    5 +-
 .../processors/cache/GridCacheProcessor.java    |   27 +-
 .../cache/GridLoadCacheCloseablePredicate.java  |   30 -
 .../distributed/dht/GridDhtCacheAdapter.java    |    6 +-
 .../GridDhtPartitionsExchangeFuture.java        |   94 +-
 .../CacheQueryCloseableScanBiPredicate.java     |   30 -
 .../cache/query/GridCacheQueryManager.java      |   52 +-
 .../CacheContinuousQueryFilterEx.java           |   31 -
 .../continuous/CacheContinuousQueryHandler.java |    5 +-
 .../platform/PlatformAwareEventFilter.java      |   39 -
 .../processors/platform/PlatformContext.java    |   37 +-
 .../platform/PlatformEventFilterListener.java   |   39 +
 .../processors/platform/PlatformException.java  |   71 +
 .../platform/PlatformExtendedException.java     |   57 +
 .../platform/PlatformLocalEventListener.java    |   28 -
 .../platform/PlatformNativeException.java       |   77 +
 .../platform/PlatformNoCallbackException.java   |   50 +
 .../cache/PlatformCacheEntryFilter.java         |   11 +-
 .../cache/PlatformCacheEntryProcessor.java      |   27 +
 .../cache/query/PlatformContinuousQuery.java    |    6 +-
 .../query/PlatformContinuousQueryFilter.java    |   30 +
 .../callback/PlatformCallbackGateway.java       |   10 +-
 .../cluster/PlatformClusterNodeFilter.java      |   28 +
 .../datastreamer/PlatformStreamReceiver.java    |   27 +
 .../platform/message/PlatformMessageFilter.java |   40 +
 .../internal/util/GridStripedSpinBusyLock.java  |  127 +
 .../cache/IgniteDynamicCacheAndNodeStop.java    |   95 +
 .../CachePutAllFailoverAbstractTest.java        |  234 ++
 .../distributed/CachePutAllFailoverTxTest.java  |   30 +
 .../testsuites/IgniteCacheTestSuite2.java       |    4 +-
 .../platform/src/main/cpp/common/Makefile.am    |   45 +
 .../platform/src/main/cpp/common/configure.ac   |   62 +
 .../src/main/cpp/common/ignite-common.pc.in     |    9 +
 .../src/main/cpp/common/include/Makefile.am     |   22 +
 .../common/include/ignite/common/concurrent.h   |  210 ++
 .../cpp/common/include/ignite/common/exports.h  |  145 ++
 .../cpp/common/include/ignite/common/java.h     |  652 ++++++
 .../cpp/common/os/linux/include/Makefile.am     |   21 +
 .../os/linux/include/ignite/common/common.h     |   81 +
 .../linux/include/ignite/common/concurrent_os.h |  394 ++++
 .../src/main/cpp/common/os/linux/src/common.cpp |   59 +
 .../cpp/common/os/linux/src/concurrent_os.cpp   |  175 ++
 .../os/win/include/ignite/common/common.h       |   56 +
 .../win/include/ignite/common/concurrent_os.h   |  406 ++++
 .../src/main/cpp/common/os/win/src/common.cpp   |   65 +
 .../cpp/common/os/win/src/concurrent_os.cpp     |  151 ++
 .../src/main/cpp/common/project/README.TXT      |    1 +
 .../src/main/cpp/common/project/vs/README.TXT   |    1 +
 .../main/cpp/common/project/vs/common.vcxproj   |  202 ++
 .../common/project/vs/common.vcxproj.filters    |   54 +
 .../src/main/cpp/common/project/vs/module.def   |   99 +
 .../src/main/cpp/common/project/vs/targetver.h  |   25 +
 .../src/main/cpp/common/src/concurrent.cpp      |   94 +
 .../src/main/cpp/common/src/exports.cpp         |  413 ++++
 .../platform/src/main/cpp/common/src/java.cpp   | 2204 ++++++++++++++++++
 .../platform/PlatformAbstractTarget.java        |   34 +-
 .../platform/PlatformContextImpl.java           |  621 +++++
 .../processors/platform/PlatformException.java  |   71 -
 .../platform/PlatformExtendedException.java     |   39 -
 .../platform/PlatformNoCallbackException.java   |   50 -
 .../platform/cache/PlatformCache.java           |   44 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    5 +-
 .../cache/PlatformCacheEntryProcessor.java      |  220 --
 .../cache/PlatformCacheEntryProcessorImpl.java  |  220 ++
 .../PlatformCachePartialUpdateException.java    |   24 +-
 .../PlatformContinuousQueryRemoteFilter.java    |   20 +-
 .../cluster/PlatformClusterNodeFilter.java      |   79 -
 .../cluster/PlatformClusterNodeFilterImpl.java  |   78 +
 .../platform/compute/PlatformAbstractTask.java  |    1 +
 .../compute/PlatformNativeException.java        |   77 -
 .../datastreamer/PlatformDataStreamer.java      |    2 +-
 .../datastreamer/PlatformStreamReceiver.java    |  120 -
 .../PlatformStreamReceiverImpl.java             |  119 +
 .../platform/events/PlatformEventFilter.java    |  164 --
 .../events/PlatformEventFilterListenerImpl.java |  163 ++
 .../platform/events/PlatformEvents.java         |   15 +-
 .../messaging/PlatformMessageFilter.java        |  110 -
 .../messaging/PlatformMessageFilterImpl.java    |  110 +
 .../messaging/PlatformMessageLocalFilter.java   |    9 +-
 .../platform/messaging/PlatformMessaging.java   |    7 +-
 .../transactions/PlatformTransactions.java      |    2 +-
 .../platform/utils/PlatformFutureUtils.java     |   82 +-
 .../platform/utils/PlatformUtils.java           |    2 +-
 parent/pom.xml                                  |   10 +
 90 files changed, 8245 insertions(+), 1382 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/57779501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/57779501/parent/pom.xml
----------------------------------------------------------------------


[11/29] ignite git commit: IGNITE-1337: Moved "common" CPP project to Ignite.

Posted by ak...@apache.org.
IGNITE-1337: Moved "common" CPP project to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 39da8534d6d99803e4b5ad60bea76655e4cbf388
Parents: 7ca8cbc
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 10:40:43 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 10:40:43 2015 +0300

----------------------------------------------------------------------
 parent/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/39da8534/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 7a2b41c..3556671 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -743,6 +743,7 @@
                                         <exclude>**/*.vcsproj</exclude>
                                         <exclude>**/*.vcsproj.filters</exclude>
                                         <exclude>**/module.def</exclude>
+                                        <exclude>**/ignite-common.pc.in</exclude>
                                     </excludes>
                                 </configuration>
                             </execution>


[17/29] ignite git commit: Improving platform interfaces.

Posted by ak...@apache.org.
Improving platform interfaces.


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

Branch: refs/heads/ignite-843
Commit: 9a5575886d667eae74871262976fd7356b0d355d
Parents: 32579d4
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 12:55:49 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 12:55:49 2015 +0300

----------------------------------------------------------------------
 .../CacheContinuousQueryFilterEx.java           | 31 --------------------
 .../continuous/CacheContinuousQueryHandler.java |  5 ++--
 .../processors/platform/PlatformContext.java    |  4 +--
 .../cache/query/PlatformContinuousQuery.java    |  6 ++--
 .../query/PlatformContinuousQueryFilter.java    | 30 +++++++++++++++++++
 .../PlatformContinuousQueryRemoteFilter.java    | 20 ++++++-------
 6 files changed, 48 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9a557588/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFilterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFilterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFilterEx.java
deleted file mode 100644
index c17433c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFilterEx.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.query.continuous;
-
-
-import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
-
-/**
- * Extended continuous query filter.
- */
-public interface CacheContinuousQueryFilterEx<K, V> extends CacheEntryEventSerializableFilter<K, V> {
-    /**
-     * Callback for query unregister event.
-     */
-    public void onQueryUnregister();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a557588/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index be3cdbd..df6b4b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDeploymentManager;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
+import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -278,8 +279,8 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
             }
 
             @Override public void onUnregister() {
-                if (rmtFilter instanceof CacheContinuousQueryFilterEx)
-                    ((CacheContinuousQueryFilterEx)rmtFilter).onQueryUnregister();
+                if (rmtFilter instanceof PlatformContinuousQueryFilter)
+                    ((PlatformContinuousQueryFilter)rmtFilter).onQueryUnregister();
             }
 
             @Override public boolean oldValueRequired() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a557588/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index 1febf07..218ae6b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -24,10 +24,10 @@ import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
+import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilter;
 import org.apache.ignite.internal.processors.platform.compute.PlatformJob;
@@ -166,7 +166,7 @@ public interface PlatformContext {
      * @param filter Native filter.
      * @return Filter.
      */
-    public CacheContinuousQueryFilterEx createContinuousQueryFilter(Object filter);
+    public PlatformContinuousQueryFilter createContinuousQueryFilter(Object filter);
 
     /**
      * Create remote message filter.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a557588/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery.java
index b7cd3c2..227af89 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery.java
@@ -17,17 +17,17 @@
 
 package org.apache.ignite.internal.processors.platform.cache.query;
 
-import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx;
 import org.apache.ignite.internal.processors.platform.PlatformTarget;
 
+import javax.cache.event.CacheEntryUpdatedListener;
+
 /**
  * Platform continuous query.
  */
-public interface PlatformContinuousQuery extends CacheEntryUpdatedListener, CacheContinuousQueryFilterEx {
+public interface PlatformContinuousQuery extends CacheEntryUpdatedListener, PlatformContinuousQueryFilter {
     /**
      * Start continuous query execution.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a557588/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryFilter.java
new file mode 100644
index 0000000..61fa137
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryFilter.java
@@ -0,0 +1,30 @@
+/*
+ * 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.platform.cache.query;
+
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+
+/**
+ * Platform continuous query filter.
+ */
+public interface PlatformContinuousQueryFilter extends CacheEntryEventSerializableFilter {
+    /**
+     * Callback for query unregister event.
+     */
+    public void onQueryUnregister();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a557588/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
index e89cae9..71aa38c 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
@@ -17,17 +17,8 @@
 
 package org.apache.ignite.internal.processors.platform.cache.query;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -35,10 +26,19 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.resources.IgniteInstanceResource;
 
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 /**
  * Continuous query filter deployed on remote nodes.
  */
-public class PlatformContinuousQueryRemoteFilter implements CacheContinuousQueryFilterEx, Externalizable {
+public class PlatformContinuousQueryRemoteFilter implements PlatformContinuousQueryFilter, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 


[26/29] ignite git commit: Added tests.

Posted by ak...@apache.org.
Added tests.


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

Branch: refs/heads/ignite-843
Commit: 88a907c08a00e60a8585806bb81acbe8c25ce2b3
Parents: 9b06cf3
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 1 17:37:42 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 1 17:37:42 2015 +0300

----------------------------------------------------------------------
 .../CachePutAllFailoverAbstractTest.java        | 234 +++++++++++++++++++
 .../distributed/CachePutAllFailoverTxTest.java  |  30 +++
 2 files changed, 264 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/88a907c0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
new file mode 100644
index 0000000..ea81680
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
@@ -0,0 +1,234 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.Serializable;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ *
+ */
+public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int NODE_CNT = 2;
+
+    /** */
+    private static final long TEST_TIME = 2 * 60_000;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return NODE_CNT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TEST_TIME + 60_000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+        ccfg.setCacheStoreFactory(null);
+        ccfg.setReadThrough(false);
+        ccfg.setWriteThrough(false);
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllFailover() throws Exception {
+        final AtomicBoolean finished = new AtomicBoolean();
+
+        final long endTime = System.currentTimeMillis() + TEST_TIME;
+
+        IgniteInternalFuture<Object> restartFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("restart-thread");
+
+                while (!finished.get() && System.currentTimeMillis() < endTime) {
+                    startGrid(NODE_CNT);
+
+                    U.sleep(500);
+
+                    stopGrid(NODE_CNT);
+                }
+
+                return null;
+            }
+        });
+
+        try {
+            final IgniteCache<TestKey, TestValue> cache = ignite(0).cache(null);
+
+            GridTestUtils.runMultiThreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    int iter = 0;
+
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    long time;
+
+                    long lastInfo = 0;
+
+                    while ((time = System.currentTimeMillis()) < endTime) {
+                        if (time - lastInfo > 5000)
+                            log.info("Do putAll [iter=" + iter + ']');
+
+                        TreeMap<TestKey, TestValue> map = new TreeMap<>();
+
+                        for (int k = 0; k < 100; k++)
+                            map.put(new TestKey(rnd.nextInt(200)), new TestValue(iter));
+
+                        cache.putAll(map);
+
+                        iter++;
+                    }
+
+                    return null;
+                }
+            }, 2, "update-thread");
+
+            finished.set(true);
+
+            restartFut.get();
+        }
+        finally {
+            finished.set(true);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestKey implements Serializable, Comparable<TestKey> {
+        /** */
+        private long key;
+
+        /**
+         * @param key Key.
+         */
+        public TestKey(long key) {
+            this.key = key;
+        }
+
+        /**
+         * @return Key.
+         */
+        public long key() {
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(@NotNull TestKey other) {
+            return ((Long)key).compareTo(other.key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestKey other = (TestKey)o;
+
+            return key == other.key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return (int)(key ^ (key >>> 32));
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestKey.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestValue implements Serializable {
+        /** */
+        private long val;
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(long val) {
+            this.val = val;
+        }
+
+        /**
+         * @return Value.
+         */
+        public long value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestValue other = (TestValue)o;
+
+            return val == other.val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/88a907c0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverTxTest.java
new file mode 100644
index 0000000..8a6f933
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverTxTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+
+/**
+ *
+ */
+public class CachePutAllFailoverTxTest extends CachePutAllFailoverAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+}


[14/29] ignite git commit: Platforms: reworked PlatformCacheEntryFilter interface.

Posted by ak...@apache.org.
Platforms: reworked PlatformCacheEntryFilter interface.


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

Branch: refs/heads/ignite-843
Commit: 980a934eb26adeaae1a358d5f9e1c6faccfcaf4a
Parents: 6d0aff4
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 11:31:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 11:31:08 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/cache/PlatformCacheEntryFilter.java     | 2 +-
 .../processors/platform/cache/PlatformCacheEntryFilterImpl.java | 5 ++---
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/980a934e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
index 94de2bc..4063c60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
@@ -22,7 +22,7 @@ import org.apache.ignite.lang.IgniteBiPredicate;
 /**
  * Platform cache entry filter interface.
  */
-public interface PlatformCacheEntryFilter<K, V> extends IgniteBiPredicate<K, V> {
+public interface PlatformCacheEntryFilter extends IgniteBiPredicate {
     /**
      * Callback invoked when filter is no longer needed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/980a934e/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
index e2d44ad..5f8ec8f 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
@@ -29,8 +29,7 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 /**
  * Interop filter. Delegates apply to native platform.
  */
-public class PlatformCacheEntryFilterImpl<K, V> extends PlatformAbstractPredicate
-    implements PlatformCacheEntryFilter<K, V> {
+public class PlatformCacheEntryFilterImpl extends PlatformAbstractPredicate implements PlatformCacheEntryFilter {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -55,7 +54,7 @@ public class PlatformCacheEntryFilterImpl<K, V> extends PlatformAbstractPredicat
     }
 
     /** {@inheritDoc} */
-    @Override public boolean apply(K k, V v) {
+    @Override public boolean apply(Object k, Object v) {
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 


[24/29] ignite git commit: IGNITE-1346: Minor opto.

Posted by ak...@apache.org.
IGNITE-1346: Minor opto.


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

Branch: refs/heads/ignite-843
Commit: 465348e161b997ff15fa7a30de300d8997133a8c
Parents: 34ce97b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 16:48:40 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 16:48:40 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/GridStripedSpinBusyLock.java     | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/465348e1/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
index 614a88f..a11b0b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
@@ -88,18 +88,20 @@ public class GridStripedSpinBusyLock {
      * Block.
      */
     public void block() {
-        boolean interrupt = false;
-
+        // 1. CAS-loop to set a writer bit.
         for (AtomicInteger state : states) {
-            // 1. CAS-loop to set a writer bit.
             while (true) {
                 int oldVal = state.get();
 
                 if (state.compareAndSet(oldVal, oldVal | WRITER_MASK))
                     break;
             }
+        }
 
-            // 2. Wait until all readers are out.
+        // 2. Wait until all readers are out.
+        boolean interrupt = false;
+
+        for (AtomicInteger state : states) {
             while (state.get() != WRITER_MASK) {
                 try {
                     Thread.sleep(10);


[07/29] ignite git commit: IGNITE-1337: Moved "common" CPP project to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7ca8cbc7/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index a9ebf84..7a2b41c 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -738,6 +738,11 @@
                                         <exclude>src/main/dotnet/Apache.Ignite.sln</exclude>
                                         <exclude>src/main/dotnet/Apache.Ignite.sln.DotSettings</exclude>
                                         <exclude>src/test/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj</exclude>
+                                        <exclude>**/Makefile.am</exclude>
+                                        <exclude>**/configure.ac</exclude>
+                                        <exclude>**/*.vcsproj</exclude>
+                                        <exclude>**/*.vcsproj.filters</exclude>
+                                        <exclude>**/module.def</exclude>
                                     </excludes>
                                 </configuration>
                             </execution>


[15/29] ignite git commit: Platforms: reworked PlatformCacheEntryProcessor interface.

Posted by ak...@apache.org.
Platforms: reworked PlatformCacheEntryProcessor interface.


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

Branch: refs/heads/ignite-843
Commit: 0e25f55cfa33ae4af7c52bd4e2ee49297ef623f4
Parents: 980a934
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 11:41:28 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 11:41:28 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |   3 +-
 .../cache/PlatformCacheEntryProcessor.java      |  27 +++
 .../cache/PlatformCacheEntryProcessor.java      | 220 -------------------
 .../cache/PlatformCacheEntryProcessorImpl.java  | 220 +++++++++++++++++++
 4 files changed, 249 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e25f55c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index 4c70360..cea8326 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilter;
@@ -248,7 +249,7 @@ public interface PlatformContext {
      * @param ptr Pointer.
      * @return Entry processor.
      */
-    public CacheEntryProcessor createCacheEntryProcessor(Object proc, long ptr);
+    public PlatformCacheEntryProcessor createCacheEntryProcessor(Object proc, long ptr);
 
     /**
      * Create cache entry filter.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e25f55c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
new file mode 100644
index 0000000..3d8022f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
@@ -0,0 +1,27 @@
+/*
+ * 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.platform.cache;
+
+import org.apache.ignite.cache.CacheEntryProcessor;
+
+/**
+ * Platform cache entry processor marker interface.
+ */
+public interface PlatformCacheEntryProcessor extends CacheEntryProcessor {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e25f55c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
deleted file mode 100644
index fd14632..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * 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.platform.cache;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheEntryProcessor;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.PlatformProcessor;
-import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-/**
- * Interop cache entry processor. Delegates processing to native platform.
- */
-public class PlatformCacheEntryProcessor<K, V, T> implements CacheEntryProcessor<K, V, T>, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Indicates that entry has not been modified  */
-    private static final byte ENTRY_STATE_INTACT = 0;
-
-    /** Indicates that entry value has been set  */
-    private static final byte ENTRY_STATE_VALUE_SET = 1;
-
-    /** Indicates that remove has been called on an entry  */
-    private static final byte ENTRY_STATE_REMOVED = 2;
-
-    /** Indicates error in processor that is written as portable.  */
-    private static final byte ENTRY_STATE_ERR_PORTABLE = 3;
-
-    /** Indicates error in processor that is written as string.  */
-    private static final byte ENTRY_STATE_ERR_STRING = 4;
-
-    /** Native portable processor */
-    private Object proc;
-
-    /** Pointer to processor in the native platform. */
-    private transient long ptr;
-
-    /**
-     * {@link java.io.Externalizable} support.
-     */
-    public PlatformCacheEntryProcessor() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param proc Native portable processor
-     * @param ptr Pointer to processor in the native platform.
-     */
-    public PlatformCacheEntryProcessor(Object proc, long ptr) {
-        this.proc = proc;
-        this.ptr = ptr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public T process(MutableEntry<K, V> entry, Object... arguments) throws EntryProcessorException {
-        try {
-            IgniteKernal ignite = (IgniteKernal)entry.unwrap(Ignite.class);
-
-            PlatformProcessor interopProc;
-
-            try {
-                interopProc = PlatformUtils.platformProcessor(ignite);
-            }
-            catch (IllegalStateException ex){
-                throw new EntryProcessorException(ex);
-            }
-
-            interopProc.awaitStart();
-
-            return execute0(interopProc.context(), entry);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
-    }
-
-    /**
-     * Executes interop entry processor on a given entry, updates entry and returns result.
-     *
-     * @param ctx Context.
-     * @param entry Entry.
-     * @return Processing result.
-     * @throws org.apache.ignite.IgniteCheckedException
-     */
-    private T execute0(PlatformContext ctx, MutableEntry<K, V> entry)
-        throws IgniteCheckedException {
-        try (PlatformMemory outMem = ctx.memory().allocate()) {
-            PlatformOutputStream out = outMem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            writeEntryAndProcessor(entry, writer);
-
-            out.synchronize();
-
-            try (PlatformMemory inMem = ctx.memory().allocate()) {
-                PlatformInputStream in = inMem.input();
-
-                ctx.gateway().cacheInvoke(outMem.pointer(), inMem.pointer());
-
-                in.synchronize();
-
-                PortableRawReaderEx reader = ctx.reader(in);
-
-                return readResultAndUpdateEntry(ctx, entry, reader);
-            }
-        }
-    }
-
-    /**
-     * Writes mutable entry and entry processor to the stream.
-     *
-     * @param entry Entry to process.
-     * @param writer Writer.
-     */
-    private void writeEntryAndProcessor(MutableEntry<K, V> entry, PortableRawWriterEx writer) {
-        writer.writeObject(entry.getKey());
-        writer.writeObject(entry.getValue());
-
-        if (ptr != 0) {
-            // Execute locally - we have a pointer to native processor.
-            writer.writeBoolean(true);
-            writer.writeLong(ptr);
-        }
-        else {
-            // We are on a remote node. Send processor holder back to native.
-            writer.writeBoolean(false);
-            writer.writeObject(proc);
-        }
-    }
-
-    /**
-     * Reads processing result from stream, updates mutable entry accordingly, and returns the result.
-     *
-     * @param entry Mutable entry to update.
-     * @param reader Reader.
-     * @return Entry processing result
-     * @throws javax.cache.processor.EntryProcessorException If processing has failed in user code.
-     */
-    @SuppressWarnings("unchecked")
-    private T readResultAndUpdateEntry(PlatformContext ctx, MutableEntry<K, V> entry, PortableRawReaderEx reader) {
-        byte state = reader.readByte();
-
-        switch (state) {
-            case ENTRY_STATE_VALUE_SET:
-                entry.setValue((V)reader.readObject());
-
-                break;
-
-            case ENTRY_STATE_REMOVED:
-                entry.remove();
-
-                break;
-
-            case ENTRY_STATE_ERR_PORTABLE:
-                // Full exception
-                Object nativeErr = reader.readObjectDetached();
-
-                assert nativeErr != null;
-
-                throw new EntryProcessorException("Failed to execute native cache entry processor.",
-                    ctx.createNativeException(nativeErr));
-
-            case ENTRY_STATE_ERR_STRING:
-                // Native exception was not serializable, we have only message.
-                String errMsg = reader.readString();
-
-                assert errMsg != null;
-
-                throw new EntryProcessorException("Failed to execute native cache entry processor: " + errMsg);
-
-            default:
-                assert state == ENTRY_STATE_INTACT;
-        }
-
-        return (T)reader.readObject();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(proc);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        proc = in.readObject();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e25f55c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
new file mode 100644
index 0000000..16124fe
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
@@ -0,0 +1,220 @@
+/*
+ * 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.platform.cache;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformProcessor;
+import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Platform cache entry processor. Delegates processing to native platform.
+ */
+public class PlatformCacheEntryProcessorImpl implements PlatformCacheEntryProcessor, Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Indicates that entry has not been modified  */
+    private static final byte ENTRY_STATE_INTACT = 0;
+
+    /** Indicates that entry value has been set  */
+    private static final byte ENTRY_STATE_VALUE_SET = 1;
+
+    /** Indicates that remove has been called on an entry  */
+    private static final byte ENTRY_STATE_REMOVED = 2;
+
+    /** Indicates error in processor that is written as portable.  */
+    private static final byte ENTRY_STATE_ERR_PORTABLE = 3;
+
+    /** Indicates error in processor that is written as string.  */
+    private static final byte ENTRY_STATE_ERR_STRING = 4;
+
+    /** Native portable processor */
+    private Object proc;
+
+    /** Pointer to processor in the native platform. */
+    private transient long ptr;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    public PlatformCacheEntryProcessorImpl() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param proc Native portable processor
+     * @param ptr Pointer to processor in the native platform.
+     */
+    public PlatformCacheEntryProcessorImpl(Object proc, long ptr) {
+        this.proc = proc;
+        this.ptr = ptr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object process(MutableEntry entry, Object... args)
+        throws EntryProcessorException {
+        try {
+            IgniteKernal ignite = (IgniteKernal)entry.unwrap(Ignite.class);
+
+            PlatformProcessor interopProc;
+
+            try {
+                interopProc = PlatformUtils.platformProcessor(ignite);
+            }
+            catch (IllegalStateException ex){
+                throw new EntryProcessorException(ex);
+            }
+
+            interopProc.awaitStart();
+
+            return execute0(interopProc.context(), entry);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * Executes interop entry processor on a given entry, updates entry and returns result.
+     *
+     * @param ctx Context.
+     * @param entry Entry.
+     * @return Processing result.
+     * @throws org.apache.ignite.IgniteCheckedException
+     */
+    private Object execute0(PlatformContext ctx, MutableEntry entry)
+        throws IgniteCheckedException {
+        try (PlatformMemory outMem = ctx.memory().allocate()) {
+            PlatformOutputStream out = outMem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writeEntryAndProcessor(entry, writer);
+
+            out.synchronize();
+
+            try (PlatformMemory inMem = ctx.memory().allocate()) {
+                PlatformInputStream in = inMem.input();
+
+                ctx.gateway().cacheInvoke(outMem.pointer(), inMem.pointer());
+
+                in.synchronize();
+
+                PortableRawReaderEx reader = ctx.reader(in);
+
+                return readResultAndUpdateEntry(ctx, entry, reader);
+            }
+        }
+    }
+
+    /**
+     * Writes mutable entry and entry processor to the stream.
+     *
+     * @param entry Entry to process.
+     * @param writer Writer.
+     */
+    private void writeEntryAndProcessor(MutableEntry entry, PortableRawWriterEx writer) {
+        writer.writeObject(entry.getKey());
+        writer.writeObject(entry.getValue());
+
+        if (ptr != 0) {
+            // Execute locally - we have a pointer to native processor.
+            writer.writeBoolean(true);
+            writer.writeLong(ptr);
+        }
+        else {
+            // We are on a remote node. Send processor holder back to native.
+            writer.writeBoolean(false);
+            writer.writeObject(proc);
+        }
+    }
+
+    /**
+     * Reads processing result from stream, updates mutable entry accordingly, and returns the result.
+     *
+     * @param entry Mutable entry to update.
+     * @param reader Reader.
+     * @return Entry processing result
+     * @throws javax.cache.processor.EntryProcessorException If processing has failed in user code.
+     */
+    @SuppressWarnings("unchecked")
+    private Object readResultAndUpdateEntry(PlatformContext ctx, MutableEntry entry, PortableRawReaderEx reader) {
+        byte state = reader.readByte();
+
+        switch (state) {
+            case ENTRY_STATE_VALUE_SET:
+                entry.setValue(reader.readObject());
+
+                break;
+
+            case ENTRY_STATE_REMOVED:
+                entry.remove();
+
+                break;
+
+            case ENTRY_STATE_ERR_PORTABLE:
+                // Full exception
+                Object nativeErr = reader.readObjectDetached();
+
+                assert nativeErr != null;
+
+                throw new EntryProcessorException("Failed to execute native cache entry processor.",
+                    ctx.createNativeException(nativeErr));
+
+            case ENTRY_STATE_ERR_STRING:
+                // Native exception was not serializable, we have only message.
+                String errMsg = reader.readString();
+
+                assert errMsg != null;
+
+                throw new EntryProcessorException("Failed to execute native cache entry processor: " + errMsg);
+
+            default:
+                assert state == ENTRY_STATE_INTACT;
+        }
+
+        return reader.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(proc);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        proc = in.readObject();
+    }
+}
\ No newline at end of file


[22/29] ignite git commit: GG-1346: Implemented striped spin busy lock.

Posted by ak...@apache.org.
GG-1346: Implemented striped spin busy lock.


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

Branch: refs/heads/ignite-843
Commit: aec0631eee9ab0e570d995b53557405748f4d891
Parents: 9b06cf3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 16:38:42 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 16:38:42 2015 +0300

----------------------------------------------------------------------
 .../callback/PlatformCallbackGateway.java       |  10 +-
 .../internal/util/GridStripedSpinBusyLock.java  | 125 +++++++++++++++++++
 2 files changed, 130 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/aec0631e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
index 93698c2..a348888 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.platform.callback;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.util.GridSpinReadWriteLock;
+import org.apache.ignite.internal.util.GridStripedSpinBusyLock;
 
 /**
  * Gateway to all platform-dependent callbacks. Implementers might extend this class and provide additional callbacks.
@@ -30,7 +30,7 @@ public class PlatformCallbackGateway {
     protected final long envPtr;
 
     /** Lock. */
-    private final GridSpinReadWriteLock lock = new GridSpinReadWriteLock();
+    private final GridStripedSpinBusyLock lock = new GridStripedSpinBusyLock();
 
     /**
      * Native gateway.
@@ -922,7 +922,7 @@ public class PlatformCallbackGateway {
      * Enter gateway.
      */
     protected void enter() {
-        if (!lock.tryReadLock())
+        if (!lock.enterBusy())
             throw new IgniteException("Failed to execute native callback because grid is stopping.");
     }
 
@@ -930,13 +930,13 @@ public class PlatformCallbackGateway {
      * Leave gateway.
      */
     protected void leave() {
-        lock.readUnlock();
+        lock.leaveBusy();
     }
 
     /**
      * Block gateway.
      */
     protected void block() {
-        lock.writeLock();
+        lock.block();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/aec0631e/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
new file mode 100644
index 0000000..678d521
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridStripedSpinBusyLock.java
@@ -0,0 +1,125 @@
+/*
+ * 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.util;
+
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Striped spin busy lock. Aimed to provide efficient "read" lock semantics while still maintaining safety when
+ * entering "busy" state.
+ */
+public class GridStripedSpinBusyLock {
+    /** Writer mask. */
+    private static int WRITER_MASK = 1 << 30;
+
+    /** Default amount of stripes. */
+    private static final int DFLT_STRIPE_CNT = Runtime.getRuntime().availableProcessors() * 4;
+
+    /** States; they are not subjects to false-sharing because actual values are located far from each other. */
+    private final AtomicInteger[] states;
+
+    /** Thread index. */
+    private static ThreadLocal<Integer> threadIdx = new ThreadLocal<Integer>() {
+        @Override protected Integer initialValue() {
+            return new Random().nextInt(Integer.MAX_VALUE);
+        }
+    };
+
+    /**
+     * Default constructor.
+     */
+    public GridStripedSpinBusyLock() {
+        this(DFLT_STRIPE_CNT);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param stripeCnt Amount of stripes.
+     */
+    public GridStripedSpinBusyLock(int stripeCnt) {
+        states = new AtomicInteger[stripeCnt];
+
+        for (int i = 0; i < stripeCnt; i++)
+            states[i] = new AtomicInteger();
+    }
+
+    /**
+     * Enter busy state.
+     *
+     * @return {@code True} if entered busy state.
+     */
+    public boolean enterBusy() {
+        int val = state().incrementAndGet();
+
+        if ((val & WRITER_MASK) == WRITER_MASK) {
+            leaveBusy();
+
+            return false;
+        }
+        else
+            return true;
+    }
+
+    /**
+     * Leave busy state.
+     */
+    public void leaveBusy() {
+        state().decrementAndGet();
+    }
+
+    /**
+     * Block.
+     */
+    public void block() {
+        boolean interrupt = false;
+
+        for (AtomicInteger state : states) {
+            // 1. CAS-loop to set a writer bit.
+            while (true) {
+                int oldVal = state.get();
+
+                if (state.compareAndSet(oldVal, oldVal | WRITER_MASK))
+                    break;
+            }
+
+            // 2. Wait until all readers are out.
+            while (state.get() != WRITER_MASK) {
+                try {
+                    Thread.sleep(10);
+                }
+                catch (InterruptedException e) {
+                    interrupt = true;
+                }
+            }
+        }
+
+        if (interrupt)
+            Thread.currentThread().interrupt();
+    }
+
+    /**
+     * Gets state of thread's stripe.
+     *
+     * @return State.
+     */
+    private AtomicInteger state() {
+        return states[threadIdx.get() % states.length];
+    }
+}


[25/29] ignite git commit: IGNITE-1341: Refactored platform exceptions.

Posted by ak...@apache.org.
IGNITE-1341: Refactored platform exceptions.


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

Branch: refs/heads/ignite-843
Commit: 66504a08c5db4e7cc03bfcb3336a00650d5f2375
Parents: 465348e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 17:15:35 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 17:15:35 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |  4 +-
 .../processors/platform/PlatformException.java  | 71 ++++++++++++++++++
 .../platform/PlatformExtendedException.java     | 57 +++++++++++++++
 .../platform/PlatformNativeException.java       | 77 ++++++++++++++++++++
 .../platform/PlatformNoCallbackException.java   | 50 +++++++++++++
 .../platform/PlatformContextImpl.java           |  4 +-
 .../processors/platform/PlatformException.java  | 71 ------------------
 .../platform/PlatformExtendedException.java     | 39 ----------
 .../platform/PlatformNoCallbackException.java   | 50 -------------
 .../platform/cache/PlatformCache.java           |  2 +-
 .../PlatformCachePartialUpdateException.java    | 15 +---
 .../platform/compute/PlatformAbstractTask.java  |  1 +
 .../compute/PlatformNativeException.java        | 77 --------------------
 .../platform/utils/PlatformUtils.java           |  2 +-
 14 files changed, 262 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index 218ae6b..a9b7d02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform;
 
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.Event;
@@ -216,8 +215,7 @@ public interface PlatformContext {
      * @param cause Native cause.
      * @return Exception.
      */
-    // TODO: Some common interface must be used here.
-    public IgniteCheckedException createNativeException(Object cause);
+    public PlatformNativeException createNativeException(Object cause);
 
     /**
      * Create job.

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java
new file mode 100644
index 0000000..4334d93
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java
@@ -0,0 +1,71 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interop checked exception.
+ */
+public class PlatformException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Create empty exception.
+     */
+    public PlatformException() {
+        // No-op.
+    }
+
+    /**
+     * Creates new exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public PlatformException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates new grid exception with given throwable as a cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public PlatformException(Throwable cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Creates new exception with given error message and optional nested exception.
+     *
+     * @param msg Error message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public PlatformException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformException.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
new file mode 100644
index 0000000..825db6c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
@@ -0,0 +1,57 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+
+/**
+ * Denotes an exception which has some data to be written in a special manner.
+ */
+public abstract class PlatformExtendedException extends PlatformException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Platform context. */
+    protected final PlatformContext ctx;
+
+    /**
+     * Constructor.
+     *
+     * @param cause Root cause.
+     * @param ctx Platform context.
+     */
+    protected PlatformExtendedException(Throwable cause, PlatformContext ctx) {
+        super(cause);
+
+        this.ctx = ctx;
+    }
+
+    /**
+     * @return Platform context.
+     */
+    public PlatformContext context() {
+        return ctx;
+    }
+
+    /**
+     * Write data.
+     *
+     * @param writer Writer.
+     */
+    public abstract void writeData(PortableRawWriterEx writer);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
new file mode 100644
index 0000000..a99664a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNativeException.java
@@ -0,0 +1,77 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Exception occurred on native side.
+ */
+public class PlatformNativeException extends PlatformException implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Native cause. */
+    protected Object cause;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public PlatformNativeException() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param cause Native cause.
+     */
+    public PlatformNativeException(Object cause) {
+        super("Native platform exception occurred.");
+
+        this.cause = cause;
+    }
+
+    /**
+     * @return Native cause.
+     */
+    public Object cause() {
+        return cause;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(cause);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        cause = in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformNativeException.class, this, "cause", cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java
new file mode 100644
index 0000000..f706bbd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java
@@ -0,0 +1,50 @@
+/*
+ * 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.platform;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Exception raised when interop callback is not set in native platform.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public class PlatformNoCallbackException extends PlatformException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Constructor.
+     */
+    public PlatformNoCallbackException() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param msg Message.
+     */
+    public PlatformNoCallbackException(String msg) {
+        super(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformNoCallbackException.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index 6d1d7a7..3895506 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform;
 
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
@@ -54,7 +53,6 @@ import org.apache.ignite.internal.processors.platform.compute.PlatformAbstractTa
 import org.apache.ignite.internal.processors.platform.compute.PlatformClosureJob;
 import org.apache.ignite.internal.processors.platform.compute.PlatformFullJob;
 import org.apache.ignite.internal.processors.platform.compute.PlatformJob;
-import org.apache.ignite.internal.processors.platform.compute.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver;
 import org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl;
 import org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl;
@@ -587,7 +585,7 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteCheckedException createNativeException(Object cause) {
+    @Override public PlatformNativeException createNativeException(Object cause) {
         return new PlatformNativeException(cause);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java
deleted file mode 100644
index 4334d93..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformException.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.platform;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Interop checked exception.
- */
-public class PlatformException extends IgniteCheckedException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Create empty exception.
-     */
-    public PlatformException() {
-        // No-op.
-    }
-
-    /**
-     * Creates new exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public PlatformException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates new grid exception with given throwable as a cause and
-     * source of error message.
-     *
-     * @param cause Non-null throwable cause.
-     */
-    public PlatformException(Throwable cause) {
-        this(cause.getMessage(), cause);
-    }
-
-    /**
-     * Creates new exception with given error message and optional nested exception.
-     *
-     * @param msg Error message.
-     * @param cause Optional nested exception (can be {@code null}).
-     */
-    public PlatformException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PlatformException.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
deleted file mode 100644
index 8c33729..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.platform;
-
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
-
-/**
- * Denotes an exception which has some data to be written in a special manner.
- */
-public interface PlatformExtendedException {
-    /**
-     * Gets platform context.
-     *
-     * @return Platform context.
-     */
-    public PlatformContext context();
-
-    /**
-     * Write data.
-     *
-     * @param writer Writer.
-     */
-    public void writeData(PortableRawWriterEx writer);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java
deleted file mode 100644
index f706bbd..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoCallbackException.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.platform;
-
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Exception raised when interop callback is not set in native platform.
- */
-@SuppressWarnings("UnusedDeclaration")
-public class PlatformNoCallbackException extends PlatformException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Constructor.
-     */
-    public PlatformNoCallbackException() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param msg Message.
-     */
-    public PlatformNoCallbackException(String msg) {
-        super(msg);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PlatformNoCallbackException.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index e579be7..184aa33 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -39,7 +39,7 @@ import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformQueryCursor;
-import org.apache.ignite.internal.processors.platform.compute.PlatformNativeException;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.GridConcurrentFactory;

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
index 58dfa4c..ef17a06 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.platform.cache;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.PlatformException;
 import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 
@@ -29,13 +28,10 @@ import java.util.Collection;
 /**
  * Interop cache partial update exception.
  */
-public class PlatformCachePartialUpdateException extends PlatformException implements PlatformExtendedException {
+public class PlatformCachePartialUpdateException extends PlatformExtendedException {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Platform context. */
-    private final PlatformContext ctx;
-
     /** Keep portable flag. */
     private final boolean keepPortable;
 
@@ -48,18 +44,11 @@ public class PlatformCachePartialUpdateException extends PlatformException imple
      */
     public PlatformCachePartialUpdateException(CachePartialUpdateCheckedException cause, PlatformContext ctx,
         boolean keepPortable) {
-        super(cause);
-
-        this.ctx = ctx;
+        super(cause, ctx);
         this.keepPortable = keepPortable;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformContext context() {
-        return ctx;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeData(PortableRawWriterEx writer) {
         Collection keys = ((CachePartialUpdateCheckedException)getCause()).failedKeys();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
index 0152713..b17dd97 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
@@ -26,6 +26,7 @@ import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformNativeException.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformNativeException.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformNativeException.java
deleted file mode 100644
index 106abb9..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformNativeException.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.platform.compute;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import org.apache.ignite.internal.processors.platform.PlatformException;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Exception occurred on native side.
- */
-public class PlatformNativeException extends PlatformException implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Native cause. */
-    protected Object cause;
-
-    /**
-     * {@link Externalizable} support.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    public PlatformNativeException() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param cause Native cause.
-     */
-    public PlatformNativeException(Object cause) {
-        super("Native platform exception occurred.");
-
-        this.cause = cause;
-    }
-
-    /**
-     * @return Native cause.
-     */
-    public Object cause() {
-        return cause;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(cause);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        cause = in.readObject();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PlatformNativeException.class, this, "cause", cause);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66504a08/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
index ab2a5eb..2e1da0b 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
@@ -36,8 +36,8 @@ import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
+import org.apache.ignite.internal.processors.platform.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
-import org.apache.ignite.internal.processors.platform.compute.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils;


[21/29] ignite git commit: IGNITE-1340: Fixes to exception propagation.

Posted by ak...@apache.org.
IGNITE-1340: Fixes to exception propagation.


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

Branch: refs/heads/ignite-843
Commit: 9b06cf3d896d17af5a132a8849df41f8822dce96
Parents: 5f58bbb
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Tue Sep 1 15:57:36 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 15:57:36 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        | 34 +++-----
 .../platform/cache/PlatformCache.java           | 42 ++++++----
 .../PlatformCachePartialUpdateException.java    |  9 ++-
 .../datastreamer/PlatformDataStreamer.java      |  2 +-
 .../transactions/PlatformTransactions.java      |  2 +-
 .../platform/utils/PlatformFutureUtils.java     | 82 ++++++++++++++++++--
 6 files changed, 116 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9b06cf3d/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 71d1657..0f46517 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -18,14 +18,14 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
-import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
@@ -171,7 +171,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @param e Exception to convert.
      * @return Converted exception.
      */
-    protected Exception convertException(Exception e) {
+    public Exception convertException(Exception e) {
         return e;
     }
 
@@ -184,12 +184,12 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
 
     /** {@inheritDoc} */
     @Override public void listenFuture(final long futId, int typ) throws Exception {
-        PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, null);
+        PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, null, this);
     }
 
     /** {@inheritDoc} */
     @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception {
-        PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, futureWriter(opId));
+        PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, futureWriter(opId), this);
     }
 
     /**
@@ -199,26 +199,10 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
-    protected IgniteFuture currentFutureWrapped() throws IgniteCheckedException {
-        return currentFuture().chain(new IgniteClosure<IgniteFuture, Object>() {
-            @Override public Object apply(IgniteFuture o) {
-                try {
-                    return o.get();
-                }
-                catch (RuntimeException e) {
-                    Exception converted = convertException(e);
-
-                    if (converted instanceof RuntimeException)
-                        throw (RuntimeException)converted;
-                    else {
-                        log.error("Interop future result cannot be obtained due to exception.", converted);
-
-                        throw new IgniteException("Interop future result cannot be obtained due to exception " +
-                            "(see log for more details).");
-                    }
-                }
-            }
-        });
+    protected IgniteInternalFuture currentFutureWrapped() throws IgniteCheckedException {
+        IgniteFutureImpl fut = (IgniteFutureImpl)currentFuture();
+
+        return fut.internalFuture();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9b06cf3d/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index a7c741e..e579be7 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -17,18 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform.cache;
 
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import javax.cache.Cache;
-import javax.cache.expiry.Duration;
-import javax.cache.expiry.ExpiryPolicy;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheEntryProcessor;
@@ -43,6 +31,7 @@ import org.apache.ignite.cache.query.TextQuery;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.cache.CacheOperationContext;
+import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
@@ -58,6 +47,19 @@ import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
+import javax.cache.Cache;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.EntryProcessorResult;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+
 /**
  * Native cache wrapper implementation.
  */
@@ -618,9 +620,16 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Exception convertException(Exception e) {
+    @Override public Exception convertException(Exception e) {
         if (e instanceof CachePartialUpdateException)
-            return new PlatformCachePartialUpdateException((CachePartialUpdateException)e, platformCtx, keepPortable);
+            return new PlatformCachePartialUpdateException((CachePartialUpdateCheckedException)e.getCause(),
+                platformCtx, keepPortable);
+
+        if (e instanceof CachePartialUpdateCheckedException)
+            return new PlatformCachePartialUpdateException((CachePartialUpdateCheckedException)e, platformCtx, keepPortable);
+
+        if (e.getCause() instanceof EntryProcessorException)
+            return (EntryProcessorException) e.getCause();
 
         return super.convertException(e);
     }
@@ -788,11 +797,10 @@ public class PlatformCache extends PlatformAbstractTarget {
      */
     public void rebalance(long futId) {
         PlatformFutureUtils.listen(platformCtx, cache.rebalance().chain(new C1<IgniteFuture, Object>() {
-            @Override
-            public Object apply(IgniteFuture fut) {
+            @Override public Object apply(IgniteFuture fut) {
                 return null;
             }
-        }), futId, PlatformFutureUtils.TYP_OBJ);
+        }), futId, PlatformFutureUtils.TYP_OBJ, this);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9b06cf3d/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
index 354cef7..58dfa4c 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
@@ -17,14 +17,15 @@
 
 package org.apache.ignite.internal.processors.platform.cache;
 
-import java.util.Collection;
-import org.apache.ignite.cache.CachePartialUpdateException;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformException;
 import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 
+import java.util.Collection;
+
 /**
  * Interop cache partial update exception.
  */
@@ -45,7 +46,7 @@ public class PlatformCachePartialUpdateException extends PlatformException imple
      * @param ctx Context.
      * @param keepPortable Keep portable flag.
      */
-    public PlatformCachePartialUpdateException(CachePartialUpdateException cause, PlatformContext ctx,
+    public PlatformCachePartialUpdateException(CachePartialUpdateCheckedException cause, PlatformContext ctx,
         boolean keepPortable) {
         super(cause);
 
@@ -60,7 +61,7 @@ public class PlatformCachePartialUpdateException extends PlatformException imple
 
     /** {@inheritDoc} */
     @Override public void writeData(PortableRawWriterEx writer) {
-        Collection keys = ((CachePartialUpdateException)getCause()).failedKeys();
+        Collection keys = ((CachePartialUpdateCheckedException)getCause()).failedKeys();
 
         writer.writeBoolean(keepPortable);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9b06cf3d/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index e0e9305..ef64ef9 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -110,7 +110,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
                             vals.add(new GridMapEntry(reader.readObjectDetached(), reader.readObjectDetached()));
 
                         PlatformFutureUtils.listen(platformCtx, ldr.addData(vals), futPtr,
-                            PlatformFutureUtils.TYP_OBJ);
+                            PlatformFutureUtils.TYP_OBJ, this);
                     }
 
                     if (plc == PLC_CLOSE) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9b06cf3d/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
index 86942c5..1d2c315 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -179,7 +179,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
             }
         });
 
-        PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ);
+        PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ, this);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9b06cf3d/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
index 59e5463..0019986 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -66,8 +68,35 @@ public class PlatformFutureUtils {
      * @param futPtr Native future pointer.
      * @param typ Expected return type.
      */
-    public static void listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr, final int typ) {
-        listen(ctx, new FutureListenable(fut), futPtr, typ, null);
+    public static void listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr, final int typ,
+        PlatformAbstractTarget target) {
+        listen(ctx, new InternalFutureListenable(fut), futPtr, typ, null, target);
+    }
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     */
+    public static void listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr, final int typ,
+        PlatformAbstractTarget target) {
+        listen(ctx, new FutureListenable(fut), futPtr, typ, null, target);
+    }
+
+    /**
+     * Listen future.
+     *
+     * @param ctx Context.
+     * @param fut Java future.
+     * @param futPtr Native future pointer.
+     * @param typ Expected return type.
+     * @param writer Writer.
+     */
+    public static void listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr, final int typ,
+        Writer writer, PlatformAbstractTarget target) {
+        listen(ctx, new InternalFutureListenable(fut), futPtr, typ, writer, target);
     }
 
     /**
@@ -80,8 +109,8 @@ public class PlatformFutureUtils {
      * @param writer Writer.
      */
     public static void listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr, final int typ,
-        Writer writer) {
-        listen(ctx, new FutureListenable(fut), futPtr, typ, writer);
+        Writer writer, PlatformAbstractTarget target) {
+        listen(ctx, new FutureListenable(fut), futPtr, typ, writer, target);
     }
 
     /**
@@ -92,8 +121,9 @@ public class PlatformFutureUtils {
      * @param futPtr Native future pointer.
      * @param writer Writer.
      */
-    public static void listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr, Writer writer) {
-        listen(ctx, new FutureListenable(fut), futPtr, TYP_OBJ, writer);
+    public static void listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr, Writer writer,
+        PlatformAbstractTarget target) {
+        listen(ctx, new InternalFutureListenable(fut), futPtr, TYP_OBJ, writer, target);
     }
 
     /**
@@ -107,13 +137,16 @@ public class PlatformFutureUtils {
      */
     @SuppressWarnings("unchecked")
     private static void listen(final PlatformContext ctx, Listenable listenable, final long futPtr, final int typ,
-        @Nullable final Writer writer) {
+        @Nullable final Writer writer, final PlatformAbstractTarget target) {
         final PlatformCallbackGateway gate = ctx.gateway();
 
         listenable.listen(new IgniteBiInClosure<Object, Throwable>() {
             private static final long serialVersionUID = 0L;
 
             @Override public void apply(Object res, Throwable err) {
+                if (err instanceof Exception)
+                    err = target.convertException((Exception)err);
+
                 if (writer != null && writeToWriter(res, err, ctx, writer, futPtr))
                     return;
 
@@ -326,4 +359,39 @@ public class PlatformFutureUtils {
             });
         }
     }
+
+    /**
+     * Listenable around Ignite future.
+     */
+    private static class InternalFutureListenable implements Listenable {
+        /** Future. */
+        private final IgniteInternalFuture fut;
+
+        /**
+         * Constructor.
+         *
+         * @param fut Future.
+         */
+        public InternalFutureListenable(IgniteInternalFuture fut) {
+            this.fut = fut;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void listen(final IgniteBiInClosure<Object, Throwable> lsnr) {
+            fut.listen(new IgniteInClosure<IgniteInternalFuture>() {
+                private static final long serialVersionUID = 0L;
+
+                @Override public void apply(IgniteInternalFuture fut0) {
+                    try {
+                        lsnr.apply(fut0.get(), null);
+                    }
+                    catch (Throwable err) {
+                        lsnr.apply(null, err);
+                    }
+                }
+            });
+        }
+    }
+
 }
\ No newline at end of file


[04/29] ignite git commit: Fixed compilation issue in Ignite.

Posted by ak...@apache.org.
Fixed compilation issue in Ignite.


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

Branch: refs/heads/ignite-843
Commit: dfc1e40eb7fb6025b0b7def972062d8aa5d689a3
Parents: afce269
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 10:29:53 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 10:29:53 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dfc1e40e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 9b9bbba..f246107 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheVariableTopologySelf
 import org.apache.ignite.internal.processors.cache.IgniteAtomicCacheEntryProcessorNodeJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorNodeJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
+import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionNotLoadedEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionedAffinityFilterSelfTest;


[16/29] ignite git commit: Improving platform interfaces.

Posted by ak...@apache.org.
Improving platform interfaces.


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

Branch: refs/heads/ignite-843
Commit: 32579d450377f177fc6078c4d9686de88dd21220
Parents: 0e25f55
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 12:49:16 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 12:49:16 2015 +0300

----------------------------------------------------------------------
 .../internal/GridEventConsumeHandler.java       |  15 +-
 .../managers/communication/GridIoManager.java   |  50 +++---
 .../GridLifecycleAwareMessageFilter.java        |  38 -----
 .../eventstorage/GridEventStorageManager.java   |  16 +-
 .../platform/PlatformAwareEventFilter.java      |  39 -----
 .../processors/platform/PlatformContext.java    |  15 +-
 .../platform/PlatformEventFilterListener.java   |  39 +++++
 .../platform/PlatformLocalEventListener.java    |  28 ----
 .../platform/message/PlatformMessageFilter.java |  40 +++++
 .../platform/events/PlatformEventFilter.java    | 164 -------------------
 .../events/PlatformEventFilterListenerImpl.java | 163 ++++++++++++++++++
 .../platform/events/PlatformEvents.java         |  15 +-
 .../messaging/PlatformMessageFilter.java        | 110 -------------
 .../messaging/PlatformMessageFilterImpl.java    | 110 +++++++++++++
 .../messaging/PlatformMessageLocalFilter.java   |   9 +-
 .../platform/messaging/PlatformMessaging.java   |   7 +-
 16 files changed, 416 insertions(+), 442 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 93e01e5..599d301 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -38,8 +38,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheDeployable;
 import org.apache.ignite.internal.processors.cache.GridCacheDeploymentManager;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
-import org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter;
-import org.apache.ignite.internal.processors.platform.PlatformLocalEventListener;
+import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.internal.util.typedef.T3;
@@ -139,8 +138,8 @@ class GridEventConsumeHandler implements GridContinuousHandler {
         if (filter != null)
             ctx.resource().injectGeneric(filter);
 
-        if (filter instanceof PlatformAwareEventFilter)
-            ((PlatformAwareEventFilter)filter).initialize(ctx);
+        if (filter instanceof PlatformEventFilterListener)
+            ((PlatformEventFilterListener)filter).initialize(ctx);
 
         final boolean loc = nodeId.equals(ctx.localNodeId());
 
@@ -260,16 +259,16 @@ class GridEventConsumeHandler implements GridContinuousHandler {
         RuntimeException err = null;
 
         try {
-            if (filter instanceof PlatformAwareEventFilter)
-                ((PlatformAwareEventFilter)filter).close();
+            if (filter instanceof PlatformEventFilterListener)
+                ((PlatformEventFilterListener)filter).onClose();
         }
         catch(RuntimeException ex) {
             err = ex;
         }
 
         try {
-            if (cb instanceof PlatformLocalEventListener)
-                ((PlatformLocalEventListener)cb).close();
+            if (cb instanceof PlatformEventFilterListener)
+                ((PlatformEventFilterListener)cb).onClose();
         }
         catch (RuntimeException ex) {
             if (err == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index aa73296..b8af8da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -17,26 +17,6 @@
 
 package org.apache.ignite.internal.managers.communication;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Queue;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentLinkedDeque;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
@@ -54,6 +34,7 @@ import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.GridSpinReadWriteLock;
@@ -83,6 +64,27 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.ConcurrentLinkedDeque8;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -1457,8 +1459,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     public void addUserMessageListener(@Nullable final Object topic, @Nullable final IgniteBiPredicate<UUID, ?> p) {
         if (p != null) {
             try {
-                if (p instanceof GridLifecycleAwareMessageFilter)
-                    ((GridLifecycleAwareMessageFilter)p).initialize(ctx);
+                if (p instanceof PlatformMessageFilter)
+                    ((PlatformMessageFilter)p).initialize(ctx);
                 else
                     ctx.resource().injectGeneric(p);
 
@@ -1795,8 +1797,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         if (lsnr instanceof GridUserMessageListener) {
             GridUserMessageListener userLsnr = (GridUserMessageListener)lsnr;
 
-            if (userLsnr.predLsnr instanceof GridLifecycleAwareMessageFilter)
-                ((GridLifecycleAwareMessageFilter)userLsnr.predLsnr).close();
+            if (userLsnr.predLsnr instanceof PlatformMessageFilter)
+                ((PlatformMessageFilter)userLsnr.predLsnr).onClose();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
deleted file mode 100644
index 2d33a65..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridLifecycleAwareMessageFilter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.managers.communication;
-
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.lang.IgniteBiPredicate;
-
-/**
- * Special version of bi-predicate for messaging with initialize/close callbacks.
- */
-public interface GridLifecycleAwareMessageFilter<K, V> extends IgniteBiPredicate<K, V> {
-    /**
-     * Initializes the filter.
-     *
-     * @param ctx Kernal context.
-     */
-    public void initialize(GridKernalContext ctx);
-
-    /**
-     * Closes the filter.
-     */
-    public void close();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
index 7b8c759..ea01e52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
@@ -47,8 +47,7 @@ import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
-import org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter;
-import org.apache.ignite.internal.processors.platform.PlatformLocalEventListener;
+import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
 import org.apache.ignite.internal.util.GridConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
@@ -681,8 +680,8 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
         {
             IgnitePredicate p = ((UserListenerWrapper)lsnr).listener();
 
-            if (p instanceof PlatformLocalEventListener)
-                ((PlatformLocalEventListener)p).close();
+            if (p instanceof PlatformEventFilterListener)
+                ((PlatformEventFilterListener)p).onClose();
         }
 
         return found;
@@ -784,19 +783,20 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
      * @param p Grid event predicate.
      * @return Collection of grid events.
      */
+    @SuppressWarnings("unchecked")
     public <T extends Event> Collection<T> localEvents(IgnitePredicate<T> p) {
         assert p != null;
 
-        if (p instanceof PlatformAwareEventFilter) {
-            PlatformAwareEventFilter p0 = (PlatformAwareEventFilter)p;
+        if (p instanceof PlatformEventFilterListener) {
+            PlatformEventFilterListener p0 = (PlatformEventFilterListener)p;
 
             p0.initialize(ctx);
 
             try {
-                return getSpi().localEvents(p0);
+                return (Collection<T>)getSpi().localEvents(p0);
             }
             finally {
-                p0.close();
+                p0.onClose();
             }
         }
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAwareEventFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAwareEventFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAwareEventFilter.java
deleted file mode 100644
index a423578..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAwareEventFilter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.platform;
-
-import java.util.UUID;
-import org.apache.ignite.events.Event;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.lang.IgniteBiPredicate;
-import org.apache.ignite.lang.IgnitePredicate;
-
-/**
- * Special version of predicate for events with initialize/close callbacks.
- */
-public interface PlatformAwareEventFilter<E extends Event> extends IgnitePredicate<E>, IgniteBiPredicate<UUID, E> {
-    /**
-     * Initializes the filter.
-     */
-    public void initialize(GridKernalContext ctx);
-
-    /**
-     * Closes the filter.
-     */
-    public void close();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index cea8326..1febf07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -18,13 +18,10 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.Event;
-import org.apache.ignite.events.EventAdapter;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx;
@@ -39,10 +36,10 @@ import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManager;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Collection;
-import java.util.UUID;
 
 /**
  * Platform context. Acts as an entry point for platform operations.
@@ -178,7 +175,7 @@ public interface PlatformContext {
      * @param ptr Pointer of deployed native filter.
      * @return Filter.
      */
-    public GridLifecycleAwareMessageFilter<UUID, Object> createRemoteMessageFilter(Object filter, long ptr);
+    public PlatformMessageFilter createRemoteMessageFilter(Object filter, long ptr);
 
     /**
      * Check whether the given event type is supported.
@@ -192,9 +189,9 @@ public interface PlatformContext {
      * Write event.
      *
      * @param writer Writer.
-     * @param event Event.
+     * @param evt Event.
      */
-    public void writeEvent(PortableRawWriterEx writer, EventAdapter event);
+    public void writeEvent(PortableRawWriterEx writer, Event evt);
 
     /**
      * Create local event filter.
@@ -202,7 +199,7 @@ public interface PlatformContext {
      * @param hnd Native handle.
      * @return Filter.
      */
-    public <E extends Event> PlatformAwareEventFilter<E> createLocalEventFilter(long hnd);
+    public PlatformEventFilterListener createLocalEventFilter(long hnd);
 
     /**
      * Create remote event filter.
@@ -211,7 +208,7 @@ public interface PlatformContext {
      * @param types Event types.
      * @return Filter.
      */
-    public <E extends Event> PlatformAwareEventFilter<E> createRemoteEventFilter(Object pred, final int... types);
+    public PlatformEventFilterListener createRemoteEventFilter(Object pred, final int... types);
 
     /**
      * Create native exception.

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformEventFilterListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformEventFilterListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformEventFilterListener.java
new file mode 100644
index 0000000..77f0ac8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformEventFilterListener.java
@@ -0,0 +1,39 @@
+/*
+ * 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.platform;
+
+import java.util.UUID;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgnitePredicate;
+
+/**
+ * Platform event filter and listener.
+ */
+public interface PlatformEventFilterListener extends IgnitePredicate<Event>, IgniteBiPredicate<UUID, Event> {
+    /**
+     * Initializes the filter.
+     */
+    public void initialize(GridKernalContext ctx);
+
+    /**
+     * Callback invoked when filter is closed.
+     */
+    public void onClose();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLocalEventListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLocalEventListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLocalEventListener.java
deleted file mode 100644
index f38d8e0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLocalEventListener.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.platform;
-
-/**
- * Special version of listener for events with close callbacks.
- */
-public interface PlatformLocalEventListener {
-    /**
-     * Closes the listener.
-     */
-    public void close();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/message/PlatformMessageFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/message/PlatformMessageFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/message/PlatformMessageFilter.java
new file mode 100644
index 0000000..e5cec0a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/message/PlatformMessageFilter.java
@@ -0,0 +1,40 @@
+/*
+ * 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.platform.message;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.lang.IgniteBiPredicate;
+
+import java.util.UUID;
+
+/**
+ * Platform message filter.
+ */
+public interface PlatformMessageFilter extends IgniteBiPredicate<UUID, Object> {
+    /**
+     * Initializes the filter.
+     *
+     * @param ctx Kernal context.
+     */
+    public void initialize(GridKernalContext ctx);
+
+    /**
+     * Closes the filter.
+     */
+    public void onClose();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilter.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilter.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilter.java
deleted file mode 100644
index 32daa1c..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilter.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.platform.events;
-
-import java.util.UUID;
-import org.apache.ignite.events.Event;
-import org.apache.ignite.events.EventAdapter;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter;
-import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.PlatformLocalEventListener;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
-
-/**
- * Platform event filter. Delegates apply to native platform.
- */
-public class PlatformEventFilter<E extends Event> implements PlatformAwareEventFilter<E>, PlatformLocalEventListener
-{
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private final Object pred;
-
-    /** Event types. */
-    private final int[] types;
-
-    /** */
-    protected transient long hnd;
-
-    /** */
-    private transient PlatformContext ctx;
-
-    /**
-     * Constructor.
-     *
-     * @param hnd Handle in the native platform.
-     * @param ctx Context.
-     */
-    public PlatformEventFilter(long hnd, PlatformContext ctx) {
-        assert ctx != null;
-        assert hnd != 0;
-
-        this.hnd = hnd;
-        this.ctx = ctx;
-
-        pred = null;
-        types = null;
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param pred .Net portable predicate.
-     */
-    public PlatformEventFilter(Object pred, final int... types) {
-        assert pred != null;
-
-        this.pred = pred;
-        this.types = types;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(E evt) {
-        return apply0(null, evt);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(UUID uuid, E evt) {
-        return apply0(uuid, evt);
-    }
-
-    /**
-     * Apply impl.
-     * @param uuid Node if.
-     * @param evt Event.
-     * @return Result.
-     */
-    private boolean apply0(final UUID uuid, final E evt) {
-        if (!ctx.isEventTypeSupported(evt.type()))
-            return false;
-
-        if (types != null) {
-            boolean match = false;
-
-            for (int type : types) {
-                if (type == evt.type()) {
-                    match = true;
-                    break;
-                }
-            }
-
-            if (!match)
-                return false;
-        }
-
-        try (PlatformMemory mem = ctx.memory().allocate()) {
-            PlatformOutputStream out = mem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            ctx.writeEvent(writer, (EventAdapter)evt);
-
-            writer.writeUuid(uuid);
-
-            out.synchronize();
-
-            int res = ctx.gateway().eventFilterApply(hnd, mem.pointer());
-
-            return res != 0;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        ctx.gateway().eventFilterDestroy(hnd);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void initialize(GridKernalContext gridCtx) {
-        ctx = PlatformUtils.platformContext(gridCtx.grid());
-
-        try (PlatformMemory mem = ctx.memory().allocate()) {
-            PlatformOutputStream out = mem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            writer.writeObjectDetached(pred);
-
-            out.synchronize();
-
-            hnd = ctx.gateway().eventFilterCreate(mem.pointer());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        return this == o || o != null && o instanceof PlatformEventFilter &&
-            hnd == ((PlatformEventFilter)o).hnd;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return (int)(hnd ^ (hnd >>> 32));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
new file mode 100644
index 0000000..b2dfd1c
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
@@ -0,0 +1,163 @@
+/*
+ * 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.platform.events;
+
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+
+import java.util.UUID;
+
+/**
+ * Platform event filter. Delegates apply to native platform.
+ */
+public class PlatformEventFilterListenerImpl implements PlatformEventFilterListener
+{
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final Object pred;
+
+    /** Event types. */
+    private final int[] types;
+
+    /** */
+    protected transient long hnd;
+
+    /** */
+    private transient PlatformContext ctx;
+
+    /**
+     * Constructor.
+     *
+     * @param hnd Handle in the native platform.
+     * @param ctx Context.
+     */
+    public PlatformEventFilterListenerImpl(long hnd, PlatformContext ctx) {
+        assert ctx != null;
+        assert hnd != 0;
+
+        this.hnd = hnd;
+        this.ctx = ctx;
+
+        pred = null;
+        types = null;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param pred .Net portable predicate.
+     */
+    public PlatformEventFilterListenerImpl(Object pred, final int... types) {
+        assert pred != null;
+
+        this.pred = pred;
+        this.types = types;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(Event evt) {
+        return apply0(null, evt);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(UUID uuid, Event evt) {
+        return apply0(uuid, evt);
+    }
+
+    /**
+     * Apply impl.
+     * @param uuid Node if.
+     * @param evt Event.
+     * @return Result.
+     */
+    private boolean apply0(final UUID uuid, final Event evt) {
+        if (!ctx.isEventTypeSupported(evt.type()))
+            return false;
+
+        if (types != null) {
+            boolean match = false;
+
+            for (int type : types) {
+                if (type == evt.type()) {
+                    match = true;
+                    break;
+                }
+            }
+
+            if (!match)
+                return false;
+        }
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            ctx.writeEvent(writer, evt);
+
+            writer.writeUuid(uuid);
+
+            out.synchronize();
+
+            int res = ctx.gateway().eventFilterApply(hnd, mem.pointer());
+
+            return res != 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        ctx.gateway().eventFilterDestroy(hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initialize(GridKernalContext gridCtx) {
+        ctx = PlatformUtils.platformContext(gridCtx.grid());
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObjectDetached(pred);
+
+            out.synchronize();
+
+            hnd = ctx.gateway().eventFilterCreate(mem.pointer());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        return this == o || o != null && o instanceof PlatformEventFilterListenerImpl &&
+            hnd == ((PlatformEventFilterListenerImpl)o).hnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return (int)(hnd ^ (hnd >>> 32));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
index fde6be5..997c019 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
@@ -22,11 +22,12 @@ import java.util.Collection;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteEvents;
+import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventAdapter;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
-import org.apache.ignite.internal.processors.platform.PlatformAwareEventFilter;
+import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
 import org.apache.ignite.internal.util.typedef.F;
@@ -205,14 +206,14 @@ public class PlatformEvents extends PlatformAbstractTarget {
 
                 boolean hasLocFilter = reader.readBoolean();
 
-                PlatformAwareEventFilter locFilter = hasLocFilter ? localFilter(reader.readLong()) : null;
+                PlatformEventFilterListener locFilter = hasLocFilter ? localFilter(reader.readLong()) : null;
 
                 boolean hasRmtFilter = reader.readBoolean();
 
                 UUID listenId;
 
                 if (hasRmtFilter) {
-                    PlatformAwareEventFilter rmtFilter = platformCtx.createRemoteEventFilter(
+                    PlatformEventFilterListener rmtFilter = platformCtx.createRemoteEventFilter(
                         reader.readObjectDetached(), readEventTypes(reader));
 
                     listenId = events.remoteListen(bufSize, interval, autoUnsubscribe, locFilter, rmtFilter);
@@ -233,16 +234,16 @@ public class PlatformEvents extends PlatformAbstractTarget {
 
                 int[] types = readEventTypes(reader);
 
-                PlatformAwareEventFilter filter = platformCtx.createRemoteEventFilter(pred, types);
+                PlatformEventFilterListener filter = platformCtx.createRemoteEventFilter(pred, types);
 
-                Collection<EventAdapter> result = events.remoteQuery(filter, timeout);
+                Collection<Event> result = events.remoteQuery(filter, timeout);
 
                 if (result == null)
                     writer.writeInt(-1);
                 else {
                     writer.writeInt(result.size());
 
-                    for (EventAdapter e : result)
+                    for (Event e : result)
                         platformCtx.writeEvent(writer, e);
                 }
 
@@ -325,7 +326,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
      * @param hnd Handle.
      * @return Interop filter.
      */
-    private PlatformAwareEventFilter localFilter(long hnd) {
+    private PlatformEventFilterListener localFilter(long hnd) {
         return platformCtx.createLocalEventFilter(hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilter.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilter.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilter.java
deleted file mode 100644
index 4237665..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilter.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.platform.messaging;
-
-import java.util.UUID;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
-import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
-
-/**
- * Interop filter. Delegates apply to native platform.
- */
-public class PlatformMessageFilter extends PlatformAbstractPredicate
-    implements GridLifecycleAwareMessageFilter<UUID, Object> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Constructor.
-     */
-    public PlatformMessageFilter()
-    {
-        super();
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param pred .Net portable predicate.
-     * @param ptr Pointer to predicate in the native platform.
-     * @param ctx Kernal context.
-     */
-    protected PlatformMessageFilter(Object pred, long ptr, PlatformContext ctx) {
-        super(pred, ptr, ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(UUID uuid, Object m) {
-        if (ptr == 0)
-            return false;  // Destroyed.
-
-        try (PlatformMemory mem = ctx.memory().allocate()) {
-            PlatformOutputStream out = mem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            writer.writeObject(uuid);
-            writer.writeObject(m);
-
-            out.synchronize();
-
-            return ctx.gateway().messagingFilterApply(ptr, mem.pointer()) != 0;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void initialize(GridKernalContext kernalCtx) {
-        if (ptr != 0)
-            return;
-
-        ctx = PlatformUtils.platformContext(kernalCtx.grid());
-
-        try (PlatformMemory mem = ctx.memory().allocate()) {
-            PlatformOutputStream out = mem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            writer.writeObject(pred);
-
-            out.synchronize();
-
-            ptr = ctx.gateway().messagingFilterCreate(mem.pointer());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        if (ptr == 0) // Already destroyed or not initialized yet.
-            return;
-
-        try {
-            assert ctx != null;
-
-            ctx.gateway().messagingFilterDestroy(ptr);
-        }
-        finally {
-            ptr = 0;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
new file mode 100644
index 0000000..1e42914
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
@@ -0,0 +1,110 @@
+/*
+ * 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.platform.messaging;
+
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+
+import java.util.UUID;
+
+/**
+ * Platform message filter. Delegates apply to native platform.
+ */
+public class PlatformMessageFilterImpl extends PlatformAbstractPredicate implements PlatformMessageFilter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Constructor.
+     */
+    public PlatformMessageFilterImpl()
+    {
+        super();
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param pred .Net portable predicate.
+     * @param ptr Pointer to predicate in the native platform.
+     * @param ctx Kernal context.
+     */
+    protected PlatformMessageFilterImpl(Object pred, long ptr, PlatformContext ctx) {
+        super(pred, ptr, ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(UUID uuid, Object m) {
+        if (ptr == 0)
+            return false;  // Destroyed.
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObject(uuid);
+            writer.writeObject(m);
+
+            out.synchronize();
+
+            return ctx.gateway().messagingFilterApply(ptr, mem.pointer()) != 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initialize(GridKernalContext kernalCtx) {
+        if (ptr != 0)
+            return;
+
+        ctx = PlatformUtils.platformContext(kernalCtx.grid());
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObject(pred);
+
+            out.synchronize();
+
+            ptr = ctx.gateway().messagingFilterCreate(mem.pointer());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        if (ptr == 0) // Already destroyed or not initialized yet.
+            return;
+
+        try {
+            assert ctx != null;
+
+            ctx.gateway().messagingFilterDestroy(ptr);
+        }
+        finally {
+            ptr = 0;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
index 8a27508..50643e1 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
@@ -17,18 +17,19 @@
 
 package org.apache.ignite.internal.processors.platform.messaging;
 
-import java.util.UUID;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
+
+import java.util.UUID;
 
 /**
  * Interop local filter. Delegates apply to native platform, uses id to identify native target.
  */
-public class PlatformMessageLocalFilter implements GridLifecycleAwareMessageFilter<UUID, Object> {
+public class PlatformMessageLocalFilter implements PlatformMessageFilter {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -71,7 +72,7 @@ public class PlatformMessageLocalFilter implements GridLifecycleAwareMessageFilt
     }
 
     /** {@inheritDoc} */
-    @Override public void close() {
+    @Override public void onClose() {
         platformCtx.gateway().messagingFilterDestroy(hnd);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32579d45/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
index 968edd5..6dfd570 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
@@ -17,17 +17,18 @@
 
 package org.apache.ignite.internal.processors.platform.messaging;
 
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter;
 import org.apache.ignite.internal.portable.PortableRawReaderEx;
 import org.apache.ignite.internal.portable.PortableRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.lang.IgniteFuture;
 
+import java.util.UUID;
+
 /**
  * Interop messaging.
  */
@@ -144,7 +145,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
 
                 Object topic = reader.readObjectDetached();
 
-                GridLifecycleAwareMessageFilter filter = platformCtx.createRemoteMessageFilter(nativeFilter, ptr);
+                PlatformMessageFilter filter = platformCtx.createRemoteMessageFilter(nativeFilter, ptr);
 
                 UUID listenId = messaging.remoteListen(topic, filter);
 


[19/29] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: 2cd3da2356bd0f9cf663886cdee2b2b22c0d6c15
Parents: fda434c 9a55758
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Sep 1 13:20:58 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Sep 1 13:20:58 2015 +0300

----------------------------------------------------------------------
 .../internal/GridEventConsumeHandler.java       |  15 +-
 .../managers/communication/GridIoManager.java   |  50 +++---
 .../GridLifecycleAwareMessageFilter.java        |  38 -----
 .../eventstorage/GridEventStorageManager.java   |  16 +-
 .../CacheContinuousQueryFilterEx.java           |  31 ----
 .../continuous/CacheContinuousQueryHandler.java |   5 +-
 .../platform/PlatformAwareEventFilter.java      |  39 -----
 .../processors/platform/PlatformContext.java    |  19 +--
 .../platform/PlatformEventFilterListener.java   |  39 +++++
 .../platform/PlatformLocalEventListener.java    |  28 ----
 .../cache/query/PlatformContinuousQuery.java    |   6 +-
 .../query/PlatformContinuousQueryFilter.java    |  30 ++++
 .../platform/message/PlatformMessageFilter.java |  40 +++++
 .../PlatformContinuousQueryRemoteFilter.java    |  20 +--
 .../platform/events/PlatformEventFilter.java    | 164 -------------------
 .../events/PlatformEventFilterListenerImpl.java | 163 ++++++++++++++++++
 .../platform/events/PlatformEvents.java         |  15 +-
 .../messaging/PlatformMessageFilter.java        | 110 -------------
 .../messaging/PlatformMessageFilterImpl.java    | 110 +++++++++++++
 .../messaging/PlatformMessageLocalFilter.java   |   9 +-
 .../platform/messaging/PlatformMessaging.java   |   7 +-
 21 files changed, 464 insertions(+), 490 deletions(-)
----------------------------------------------------------------------



[03/29] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into master-main

Posted by ak...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into master-main


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

Branch: refs/heads/ignite-843
Commit: 1eded2506f79a40a0e67721ff3924b7edd023179
Parents: 8438d0a afce269
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Sep 1 09:18:17 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Sep 1 09:18:17 2015 +0300

----------------------------------------------------------------------
 .../examples/datagrid/CacheQueryExample.java    |  6 +-
 .../hibernate/HibernateL2CacheExample.java      |  4 +-
 .../java8/datagrid/CacheAffinityExample.java    |  4 +-
 .../java8/datagrid/CacheApiExample.java         |  4 +-
 .../java8/datagrid/CacheAsyncApiExample.java    |  4 +-
 .../processors/cache/GridCacheProcessor.java    | 27 ++++--
 .../GridDhtPartitionsExchangeFuture.java        | 94 +++++++++++--------
 .../cache/IgniteDynamicCacheAndNodeStop.java    | 95 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  3 +-
 9 files changed, 186 insertions(+), 55 deletions(-)
----------------------------------------------------------------------



[06/29] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-843
Commit: 10e51d1665e7682f12079defa452ff5aee9f0d37
Parents: 16db330 1eded25
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 10:30:25 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 10:30:25 2015 +0300

----------------------------------------------------------------------
 idea/ignite_codeStyle.xml | 11 ++---------
 1 file changed, 2 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[12/29] ignite git commit: Platforms: WIP on better interfaces.

Posted by ak...@apache.org.
Platforms: WIP on better interfaces.


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

Branch: refs/heads/ignite-843
Commit: 66d46ecacd0a6babce6e0a722c580d6646eeb9cb
Parents: 39da853
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Sep 1 11:23:50 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Sep 1 11:23:50 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |  13 +-
 .../cluster/PlatformClusterNodeFilter.java      |  28 +++++
 .../datastreamer/PlatformStreamReceiver.java    |  27 +++++
 .../cluster/PlatformClusterNodeFilter.java      |  79 ------------
 .../cluster/PlatformClusterNodeFilterImpl.java  |  78 ++++++++++++
 .../datastreamer/PlatformStreamReceiver.java    | 120 -------------------
 .../PlatformStreamReceiverImpl.java             | 119 ++++++++++++++++++
 7 files changed, 259 insertions(+), 205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index bff0fc8..4c70360 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.platform;
 
-import java.util.Collection;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cluster.ClusterMetrics;
@@ -33,15 +31,18 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinu
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
+import org.apache.ignite.internal.processors.platform.cluster.PlatformClusterNodeFilter;
 import org.apache.ignite.internal.processors.platform.compute.PlatformJob;
+import org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManager;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.stream.StreamReceiver;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.Collection;
+import java.util.UUID;
+
 /**
  * Platform context. Acts as an entry point for platform operations.
  */
@@ -266,7 +267,7 @@ public interface PlatformContext {
      * @param keepPortable Keep portable flag.
      * @return Stream receiver.
      */
-    public StreamReceiver createStreamReceiver(Object rcv, long ptr, boolean keepPortable);
+    public PlatformStreamReceiver createStreamReceiver(Object rcv, long ptr, boolean keepPortable);
 
     /**
      * Create cluster node filter.
@@ -274,5 +275,5 @@ public interface PlatformContext {
      * @param filter Native filter.
      * @return Cluster node filter.
      */
-    public IgnitePredicate<ClusterNode> createClusterNodeFilter(Object filter);
+    public PlatformClusterNodeFilter createClusterNodeFilter(Object filter);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java
new file mode 100644
index 0000000..fd550fb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java
@@ -0,0 +1,28 @@
+/*
+ * 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.platform.cluster;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.lang.IgnitePredicate;
+
+/**
+ * Platform cluster node filter marker interface.
+ */
+public interface PlatformClusterNodeFilter extends IgnitePredicate<ClusterNode> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
new file mode 100644
index 0000000..9108920
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
@@ -0,0 +1,27 @@
+/*
+ * 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.platform.datastreamer;
+
+import org.apache.ignite.stream.StreamReceiver;
+
+/**
+ * Platform data streamer filter marker interface.
+ */
+public interface PlatformStreamReceiver extends StreamReceiver<Object, Object> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java
deleted file mode 100644
index eb203cd..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.platform.cluster;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
-import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
-import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.resources.IgniteInstanceResource;
-
-/**
- * Interop cluster node filter.
- */
-public class PlatformClusterNodeFilter extends PlatformAbstractPredicate implements IgnitePredicate<ClusterNode> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * {@link java.io.Externalizable} support.
-     */
-    public PlatformClusterNodeFilter() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param pred .Net portable predicate.
-     * @param ctx Kernal context.
-     */
-    public PlatformClusterNodeFilter(Object pred, PlatformContext ctx) {
-        super(pred, 0, ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean apply(ClusterNode clusterNode) {
-        try (PlatformMemory mem = ctx.memory().allocate()) {
-            PlatformOutputStream out = mem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            writer.writeObject(pred);
-            ctx.writeNode(writer, clusterNode);
-
-            out.synchronize();
-
-            return ctx.gateway().clusterNodeFilterApply(mem.pointer()) != 0;
-        }
-    }
-
-    /**
-     * @param ignite Ignite instance.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    @IgniteInstanceResource
-    public void setIgniteInstance(Ignite ignite) {
-        ctx = PlatformUtils.platformContext(ignite);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
new file mode 100644
index 0000000..5ba9a85
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
@@ -0,0 +1,78 @@
+/*
+ * 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.platform.cluster;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+/**
+ * Interop cluster node filter.
+ */
+public class PlatformClusterNodeFilterImpl extends PlatformAbstractPredicate implements PlatformClusterNodeFilter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    public PlatformClusterNodeFilterImpl() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param pred .Net portable predicate.
+     * @param ctx Kernal context.
+     */
+    public PlatformClusterNodeFilterImpl(Object pred, PlatformContext ctx) {
+        super(pred, 0, ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(ClusterNode clusterNode) {
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObject(pred);
+            ctx.writeNode(writer, clusterNode);
+
+            out.synchronize();
+
+            return ctx.gateway().clusterNodeFilterApply(mem.pointer()) != 0;
+        }
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    @IgniteInstanceResource
+    public void setIgniteInstance(Ignite ignite) {
+        ctx = PlatformUtils.platformContext(ignite);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
deleted file mode 100644
index 851216a..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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.platform.datastreamer;
-
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Collection;
-import java.util.Map;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
-import org.apache.ignite.internal.processors.platform.PlatformContext;
-import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
-import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.stream.StreamReceiver;
-
-/**
- * Interop receiver.
- */
-public class PlatformStreamReceiver<K, V> extends PlatformAbstractPredicate implements StreamReceiver<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private boolean keepPortable;
-
-    /**
-     * Constructor.
-     */
-    public PlatformStreamReceiver()
-    {
-        super();
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param pred .Net portable receiver.
-     * @param ptr Pointer to receiver in the native platform.
-     * @param ctx Kernal context.
-     */
-    public PlatformStreamReceiver(Object pred, long ptr, boolean keepPortable, PlatformContext ctx) {
-        super(pred, ptr, ctx);
-
-        assert pred != null;
-
-        this.keepPortable = keepPortable;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void receive(IgniteCache<K, V> cache, Collection<Map.Entry<K, V>> collection)
-        throws IgniteException {
-        assert ctx != null;
-
-        try (PlatformMemory mem = ctx.memory().allocate()) {
-            PlatformOutputStream out = mem.output();
-
-            PortableRawWriterEx writer = ctx.writer(out);
-
-            writer.writeObject(pred);
-
-            writer.writeInt(collection.size());
-
-            for (Map.Entry<K, V> e : collection) {
-                writer.writeObject(e.getKey());
-                writer.writeObject(e.getValue());
-            }
-
-            out.synchronize();
-
-            ctx.gateway().dataStreamerStreamReceiverInvoke(ptr,
-                new PlatformCache(ctx, cache, keepPortable), mem.pointer(), keepPortable);
-        }
-    }
-
-    /**
-     * @param ignite Ignite instance.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    @IgniteInstanceResource
-    public void setIgniteInstance(Ignite ignite) {
-        ctx = PlatformUtils.platformContext(ignite);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        super.writeExternal(out);
-
-        out.writeBoolean(keepPortable);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        super.readExternal(in);
-
-        keepPortable = in.readBoolean();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/66d46eca/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
new file mode 100644
index 0000000..92250c0
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
@@ -0,0 +1,119 @@
+/*
+ * 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.platform.datastreamer;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Interop receiver.
+ */
+public class PlatformStreamReceiverImpl extends PlatformAbstractPredicate implements PlatformStreamReceiver {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private boolean keepPortable;
+
+    /**
+     * Constructor.
+     */
+    public PlatformStreamReceiverImpl()
+    {
+        super();
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param pred .Net portable receiver.
+     * @param ptr Pointer to receiver in the native platform.
+     * @param ctx Kernal context.
+     */
+    public PlatformStreamReceiverImpl(Object pred, long ptr, boolean keepPortable, PlatformContext ctx) {
+        super(pred, ptr, ctx);
+
+        assert pred != null;
+
+        this.keepPortable = keepPortable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void receive(IgniteCache<Object, Object> cache, Collection<Map.Entry<Object, Object>> collection)
+        throws IgniteException {
+        assert ctx != null;
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObject(pred);
+
+            writer.writeInt(collection.size());
+
+            for (Map.Entry<Object, Object> e : collection) {
+                writer.writeObject(e.getKey());
+                writer.writeObject(e.getValue());
+            }
+
+            out.synchronize();
+
+            ctx.gateway().dataStreamerStreamReceiverInvoke(ptr, new PlatformCache(ctx, cache, keepPortable),
+                mem.pointer(), keepPortable);
+        }
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    @IgniteInstanceResource
+    public void setIgniteInstance(Ignite ignite) {
+        ctx = PlatformUtils.platformContext(ignite);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        out.writeBoolean(keepPortable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        keepPortable = in.readBoolean();
+    }
+}
\ No newline at end of file