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/08/31 04:00:44 UTC

[01/50] [abbrv] ignite git commit: IGNITE-1308: Moved regular (not continuous!) queries to Ignite.

Repository: ignite
Updated Branches:
  refs/heads/ignite-843 231a88c92 -> 331950c75


IGNITE-1308: Moved regular (not continuous!) queries to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 8529e10855e71c63c4fc5a83a9cdb2300109bb19
Parents: 975f47e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 15:43:34 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 15:43:34 2015 +0300

----------------------------------------------------------------------
 .../query/PlatformAbstractQueryCursor.java      | 192 +++++++++++++++++++
 .../cache/query/PlatformFieldsQueryCursor.java  |  50 +++++
 .../cache/query/PlatformQueryCursor.java        |  46 +++++
 3 files changed, 288 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8529e108/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
new file mode 100644
index 0000000..cdd29fd
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
@@ -0,0 +1,192 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTarget implements AutoCloseable {
+    /** Get multiple entries. */
+    private static final int OP_GET_ALL = 1;
+
+    /** Get all entries. */
+    private static final int OP_GET_BATCH = 2;
+
+    /** Get single entry. */
+    private static final int OP_GET_SINGLE = 3;
+
+    /** Underlying cursor. */
+    private final QueryCursorEx<T> cursor;
+
+    /** Batch size size. */
+    private final int batchSize;
+
+    /** Underlying iterator. */
+    private Iterator<T> iter;
+
+    /**
+     * Constructor.
+     *
+     * @param interopCtx Interop context.
+     * @param cursor Underlying cursor.
+     * @param batchSize Batch size.
+     */
+    public PlatformAbstractQueryCursor(PlatformContext interopCtx, QueryCursorEx<T> cursor, int batchSize) {
+        super(interopCtx);
+
+        this.cursor = cursor;
+        this.batchSize = batchSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutOp(int type, final PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_BATCH: {
+                assert iter != null : "iterator() has not been called";
+
+                try {
+                    int cntPos = writer.reserveInt();
+
+                    int cnt;
+
+                    for (cnt = 0; cnt < batchSize; cnt++) {
+                        if (iter.hasNext())
+                            write(writer, iter.next());
+                        else
+                            break;
+                    }
+
+                    writer.writeInt(cntPos, cnt);
+                }
+                catch (Exception err) {
+                    throw PlatformUtils.unwrapQueryException(err);
+                }
+
+                break;
+            }
+
+            case OP_GET_SINGLE: {
+                assert iter != null : "iterator() has not been called";
+
+                try {
+                    if (iter.hasNext()) {
+                        write(writer, iter.next());
+
+                        return;
+                    }
+                }
+                catch (Exception err) {
+                    throw PlatformUtils.unwrapQueryException(err);
+                }
+
+                throw new IgniteCheckedException("No more data available.");
+            }
+
+            case OP_GET_ALL: {
+                try {
+                    int pos = writer.reserveInt();
+
+                    Consumer<T> consumer = new Consumer<>(this, writer);
+
+                    cursor.getAll(consumer);
+
+                    writer.writeInt(pos, consumer.cnt);
+                }
+                catch (Exception err) {
+                    throw PlatformUtils.unwrapQueryException(err);
+                }
+
+                break;
+            }
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /**
+     * Get cursor iterator.
+     */
+    public void iterator() {
+        iter = cursor.iterator();
+    }
+
+    /**
+     * Check whether next iterator entry exists.
+     *
+     * @return {@code True} if exists.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public boolean iteratorHasNext() {
+        assert iter != null : "iterator() has not been called";
+
+        return iter.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws Exception {
+        cursor.close();
+    }
+
+    /**
+     * Write value to the stream. Extension point to perform conversions on the object before writing it.
+     *
+     * @param writer Writer.
+     * @param val Value.
+     */
+    protected abstract void write(PortableRawWriterEx writer, T val);
+
+    /**
+     * Query cursor consumer.
+     */
+    private static class Consumer<T> implements QueryCursorEx.Consumer<T> {
+        /** Current query cursor. */
+        private final PlatformAbstractQueryCursor<T> cursor;
+
+        /** Writer. */
+        private final PortableRawWriterEx writer;
+
+        /** Count. */
+        private int cnt;
+
+        /**
+         * Constructor.
+         *
+         * @param writer Writer.
+         */
+        public Consumer(PlatformAbstractQueryCursor<T> cursor, PortableRawWriterEx writer) {
+            this.cursor = cursor;
+            this.writer = writer;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void consume(T val) throws IgniteCheckedException {
+            cursor.write(writer, val);
+
+            cnt++;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8529e108/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
new file mode 100644
index 0000000..f18a79a
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.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.cache.query;
+
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.platform.*;
+
+import java.util.*;
+
+/**
+ * Interop cursor for fields query.
+ */
+public class PlatformFieldsQueryCursor extends PlatformAbstractQueryCursor<List<?>> {
+    /**
+     * Constructor.
+     *
+     * @param interopCtx Interop context.
+     * @param cursor Backing cursor.
+     * @param batchSize Batch size.
+     */
+    public PlatformFieldsQueryCursor(PlatformContext interopCtx, QueryCursorEx<List<?>> cursor, int batchSize) {
+        super(interopCtx, cursor, batchSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void write(PortableRawWriterEx writer, List vals) {
+        assert vals != null;
+
+        writer.writeInt(vals.size());
+
+        for (Object val : vals)
+            writer.writeObjectDetached(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8529e108/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
new file mode 100644
index 0000000..cc96d6f
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
@@ -0,0 +1,46 @@
+/*
+ * 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.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.platform.*;
+
+import javax.cache.*;
+
+/**
+ * Interop cursor for regular queries.
+ */
+public class PlatformQueryCursor extends PlatformAbstractQueryCursor<Cache.Entry> {
+    /**
+     * Constructor.
+     *
+     * @param interopCtx Interop context.
+     * @param cursor Backing cursor.
+     * @param batchSize Batch size.
+     */
+    public PlatformQueryCursor(PlatformContext interopCtx, QueryCursorEx<Cache.Entry> cursor, int batchSize) {
+        super(interopCtx, cursor, batchSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void write(PortableRawWriterEx writer, Cache.Entry val) {
+        writer.writeObjectDetached(val.getKey());
+        writer.writeObjectDetached(val.getValue());
+    }
+}


[33/50] [abbrv] 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/c279fca9
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c279fca9
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c279fca9

Branch: refs/heads/ignite-843
Commit: c279fca96e16b54482725bbca4d9a30b33144c33
Parents: 28c8dc7 b132006
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Aug 27 13:58:34 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Aug 27 13:58:34 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtTxPrepareFuture.java          | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[44/50] [abbrv] ignite git commit: IGNITE-1317: Moved platform cache to Ignite.

Posted by ak...@apache.org.
IGNITE-1317: Moved platform cache to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 207b6820778e0d65c8181d8094903cc1dd82a863
Parents: 26f0ee0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 14:36:29 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 14:36:29 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |   22 +-
 .../cache/PlatformCacheEntryFilter.java         |   29 +
 .../platform/cache/PlatformCache.java           | 1056 ++++++++++++++++++
 .../cache/PlatformCacheEntryFilterImpl.java     |  105 ++
 .../cache/PlatformCacheEntryProcessor.java      |  212 ++++
 5 files changed, 1423 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/207b6820/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 5275e0d..cbcc91b 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,12 +18,14 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.portable.*;
 import org.apache.ignite.internal.processors.cache.query.continuous.*;
+import org.apache.ignite.internal.processors.platform.cache.*;
 import org.apache.ignite.internal.processors.platform.cache.query.*;
 import org.apache.ignite.internal.processors.platform.callback.*;
 import org.apache.ignite.internal.processors.platform.compute.*;
@@ -223,10 +225,28 @@ public interface PlatformContext {
     /**
      * Create closure job.
      *
-     * @param task Task.
+     * @param task Native task.
      * @param ptr Pointer.
      * @param job Native job.
      * @return Closure job.
      */
     public PlatformJob createClosureJob(Object task, long ptr, Object job);
+
+    /**
+     * Create cache entry processor.
+     *
+     * @param proc Native processor.
+     * @param ptr Pointer.
+     * @return Entry processor.
+     */
+    public CacheEntryProcessor createCacheEntryProcessor(Object proc, long ptr);
+
+    /**
+     * Create cache entry filter.
+     *
+     * @param filter Native filter.
+     * @param ptr Pointer.
+     * @return Entry filter.
+     */
+    public PlatformCacheEntryFilter createCacheEntryFilter(Object filter, long ptr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/207b6820/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
new file mode 100644
index 0000000..ac7cba4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilter.java
@@ -0,0 +1,29 @@
+/*
+ * 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.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+
+/**
+ * Platform cache entry filter interface.
+ */
+public interface PlatformCacheEntryFilter<K, V> extends GridLoadCacheCloseablePredicate<K, V>,
+    CacheQueryCloseableScanBiPredicate<K, V> {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/207b6820/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
new file mode 100644
index 0000000..dff9d67
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -0,0 +1,1056 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.cache.query.*;
+import org.apache.ignite.internal.processors.platform.compute.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.expiry.*;
+import javax.cache.processor.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Native cache wrapper implementation.
+ */
+@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources"})
+public class PlatformCache extends PlatformAbstractTarget {
+    /** */
+    public static final int OP_CLEAR = 1;
+
+    /** */
+    public static final int OP_CLEAR_ALL = 2;
+
+    /** */
+    public static final int OP_CONTAINS_KEY = 3;
+
+    /** */
+    public static final int OP_CONTAINS_KEYS = 4;
+
+    /** */
+    public static final int OP_GET = 5;
+
+    /** */
+    public static final int OP_GET_ALL = 6;
+
+    /** */
+    public static final int OP_GET_AND_PUT = 7;
+
+    /** */
+    public static final int OP_GET_AND_PUT_IF_ABSENT = 8;
+
+    /** */
+    public static final int OP_GET_AND_REMOVE = 9;
+
+    /** */
+    public static final int OP_GET_AND_REPLACE = 10;
+
+    /** */
+    public static final int OP_GET_NAME = 11;
+
+    /** */
+    public static final int OP_INVOKE = 12;
+
+    /** */
+    public static final int OP_INVOKE_ALL = 13;
+
+    /** */
+    public static final int OP_IS_LOCAL_LOCKED = 14;
+
+    /** */
+    public static final int OP_LOAD_CACHE = 15;
+
+    /** */
+    public static final int OP_LOC_EVICT = 16;
+
+    /** */
+    public static final int OP_LOC_LOAD_CACHE = 17;
+
+    /** */
+    public static final int OP_LOC_PROMOTE = 18;
+
+    /** */
+    public static final int OP_LOCAL_CLEAR = 20;
+
+    /** */
+    public static final int OP_LOCAL_CLEAR_ALL = 21;
+
+    /** */
+    public static final int OP_LOCK = 22;
+
+    /** */
+    public static final int OP_LOCK_ALL = 23;
+
+    /** */
+    public static final int OP_METRICS = 24;
+
+    /** */
+    private static final int OP_PEEK = 25;
+
+    /** */
+    private static final int OP_PUT = 26;
+
+    /** */
+    private static final int OP_PUT_ALL = 27;
+
+    /** */
+    public static final int OP_PUT_IF_ABSENT = 28;
+
+    /** */
+    public static final int OP_QRY_CONTINUOUS = 29;
+
+    /** */
+    public static final int OP_QRY_SCAN = 30;
+
+    /** */
+    public static final int OP_QRY_SQL = 31;
+
+    /** */
+    public static final int OP_QRY_SQL_FIELDS = 32;
+
+    /** */
+    public static final int OP_QRY_TXT = 33;
+
+    /** */
+    public static final int OP_REMOVE_ALL = 34;
+
+    /** */
+    public static final int OP_REMOVE_BOOL = 35;
+
+    /** */
+    public static final int OP_REMOVE_OBJ = 36;
+
+    /** */
+    public static final int OP_REPLACE_2 = 37;
+
+    /** */
+    public static final int OP_REPLACE_3 = 38;
+
+    /** Underlying JCache. */
+    private final IgniteCacheProxy cache;
+
+    /** Whether this cache is created with "keepPortable" flag on the other side. */
+    private final boolean keepPortable;
+
+    /** */
+    private static final GetAllWriter WRITER_GET_ALL = new GetAllWriter();
+
+    /** */
+    private static final EntryProcessorExceptionWriter WRITER_PROC_ERR = new EntryProcessorExceptionWriter();
+
+    /** */
+    private static final EntryProcessorResultsWriter WRITER_INVOKE_ALL = new EntryProcessorResultsWriter();
+
+    /** Map with currently active locks. */
+    private final ConcurrentMap<Long, Lock> lockMap = GridConcurrentFactory.newMap();
+
+    /** Lock ID sequence. */
+    private static final AtomicLong LOCK_ID_GEN = new AtomicLong();
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param cache Underlying cache.
+     * @param keepPortable Keep portable flag.
+     */
+    public PlatformCache(PlatformContext platformCtx, IgniteCache cache, boolean keepPortable) {
+        super(platformCtx);
+
+        this.cache = (IgniteCacheProxy)cache;
+        this.keepPortable = keepPortable;
+    }
+
+    /**
+     * Gets cache with "skip-store" flag set.
+     *
+     * @return Cache with "skip-store" flag set.
+     */
+    public PlatformCache withSkipStore() {
+        if (cache.delegate().skipStore())
+            return this;
+
+        return new PlatformCache(platformCtx, cache.withSkipStore(), keepPortable);
+    }
+
+    /**
+     * Gets cache with "keep portable" flag.
+     *
+     * @return Cache with "keep portable" flag set.
+     */
+    public PlatformCache withKeepPortable() {
+        if (keepPortable)
+            return this;
+
+        return new PlatformCache(platformCtx, cache.withSkipStore(), true);
+    }
+
+    /**
+     * Gets cache with provided expiry policy.
+     *
+     * @param create Create.
+     * @param update Update.
+     * @param access Access.
+     * @return Cache.
+     */
+    public PlatformCache withExpiryPolicy(final long create, final long update, final long access) {
+        IgniteCache cache0 = cache.withExpiryPolicy(new InteropExpiryPolicy(create, update, access));
+
+        return new PlatformCache(platformCtx, cache0, keepPortable);
+    }
+
+    /**
+     * Gets cache with asynchronous mode enabled.
+     *
+     * @return Cache with asynchronous mode enabled.
+     */
+    public PlatformCache withAsync() {
+        if (cache.isAsync())
+            return this;
+
+        return new PlatformCache(platformCtx, (IgniteCache)cache.withAsync(), keepPortable);
+    }
+
+    /**
+     * Gets cache with no-retries mode enabled.
+     *
+     * @return Cache with no-retries mode enabled.
+     */
+    public PlatformCache withNoRetries() {
+        CacheOperationContext opCtx = cache.operationContext();
+
+        if (opCtx != null && opCtx.noRetries())
+            return this;
+
+        return new PlatformCache(platformCtx, cache.withNoRetries(), keepPortable);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_PUT:
+                cache.put(reader.readObjectDetached(), reader.readObjectDetached());
+
+                return TRUE;
+
+            case OP_REMOVE_BOOL:
+                return cache.remove(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE;
+
+            case OP_REMOVE_ALL:
+                cache.removeAll(PlatformUtils.readSet(reader));
+
+                return TRUE;
+
+            case OP_PUT_ALL:
+                cache.putAll(PlatformUtils.readMap(reader));
+
+                return TRUE;
+
+            case OP_LOC_EVICT:
+                cache.localEvict(PlatformUtils.readCollection(reader));
+
+                return TRUE;
+
+            case OP_CONTAINS_KEY:
+                return cache.containsKey(reader.readObjectDetached()) ? TRUE : FALSE;
+
+            case OP_CONTAINS_KEYS:
+                return cache.containsKeys(PlatformUtils.readSet(reader)) ? TRUE : FALSE;
+
+            case OP_LOC_PROMOTE: {
+                cache.localPromote(PlatformUtils.readSet(reader));
+
+                break;
+            }
+
+            case OP_REPLACE_3:
+                return cache.replace(reader.readObjectDetached(), reader.readObjectDetached(),
+                    reader.readObjectDetached()) ? TRUE : FALSE;
+
+            case OP_LOC_LOAD_CACHE:
+                loadCache0(reader, true);
+
+                break;
+
+            case OP_LOAD_CACHE:
+                loadCache0(reader, false);
+
+                break;
+
+            case OP_CLEAR:
+                cache.clear(reader.readObjectDetached());
+
+                break;
+
+            case OP_CLEAR_ALL:
+                cache.clearAll(PlatformUtils.readSet(reader));
+
+                break;
+
+            case OP_LOCAL_CLEAR:
+                cache.localClear(reader.readObjectDetached());
+
+                break;
+
+            case OP_LOCAL_CLEAR_ALL:
+                cache.localClearAll(PlatformUtils.readSet(reader));
+
+                break;
+
+            case OP_PUT_IF_ABSENT: {
+                return cache.putIfAbsent(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE;
+            }
+
+            case OP_REPLACE_2: {
+                return cache.replace(reader.readObjectDetached(), reader.readObjectDetached()) ? TRUE : FALSE;
+            }
+
+            case OP_REMOVE_OBJ: {
+                return cache.remove(reader.readObjectDetached()) ? TRUE : FALSE;
+            }
+
+            case OP_IS_LOCAL_LOCKED:
+                return cache.isLocalLocked(reader.readObjectDetached(), reader.readBoolean()) ? TRUE : FALSE;
+
+            default:
+                throw new IgniteCheckedException("Unsupported operation type: " + type);
+        }
+
+        return TRUE;
+    }
+
+    /**
+     * Loads cache via localLoadCache or loadCache.
+     */
+    private void loadCache0(PortableRawReaderEx reader, boolean loc) throws IgniteCheckedException {
+        PlatformCacheEntryFilter filter = null;
+
+        Object pred = reader.readObjectDetached();
+
+        if (pred != null)
+            filter = platformCtx.createCacheEntryFilter(pred, reader.readLong());
+
+        Object[] args = reader.readObjectArray();
+
+        if (loc)
+            cache.localLoadCache(filter, args);
+        else
+            cache.loadCache(filter, args);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object processInOpObject(int type, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
+        switch (type) {
+            case OP_QRY_SQL:
+                return runQuery(reader, readSqlQuery(reader));
+
+            case OP_QRY_SQL_FIELDS:
+                return runFieldsQuery(reader, readFieldsQuery(reader));
+
+            case OP_QRY_TXT:
+                return runQuery(reader, readTextQuery(reader));
+
+            case OP_QRY_SCAN:
+                return runQuery(reader, readScanQuery(reader));
+
+            case OP_QRY_CONTINUOUS: {
+                long ptr = reader.readLong();
+                boolean loc = reader.readBoolean();
+                boolean hasFilter = reader.readBoolean();
+                Object filter = reader.readObjectDetached();
+                int bufSize = reader.readInt();
+                long timeInterval = reader.readLong();
+                boolean autoUnsubscribe = reader.readBoolean();
+                Query initQry = readInitialQuery(reader);
+
+                PlatformContinuousQuery qry = platformCtx.createContinuousQuery(ptr, hasFilter, filter);
+
+                qry.start(cache, loc, bufSize, timeInterval, autoUnsubscribe, initQry);
+
+                return qry;
+            }
+
+            default:
+                return throwUnsupported(type);
+        }
+    }
+
+    /**
+     * Read arguments for SQL query.
+     *
+     * @param reader Reader.
+     * @return Arguments.
+     */
+    @Nullable private Object[] readQueryArgs(PortableRawReaderEx reader) {
+        int cnt = reader.readInt();
+
+        if (cnt > 0) {
+            Object[] args = new Object[cnt];
+
+            for (int i = 0; i < cnt; i++)
+                args[i] = reader.readObjectDetached();
+
+            return args;
+        }
+        else
+            return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutOp(int type, PortableRawWriterEx w) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_NAME:
+                w.writeObject(cache.getName());
+
+                break;
+
+            case OP_METRICS:
+                CacheMetrics metrics = cache.metrics();
+
+                w.writeLong(metrics.getCacheGets());
+                w.writeLong(metrics.getCachePuts());
+                w.writeLong(metrics.getCacheHits());
+                w.writeLong(metrics.getCacheMisses());
+                w.writeLong(metrics.getCacheTxCommits());
+                w.writeLong(metrics.getCacheTxRollbacks());
+                w.writeLong(metrics.getCacheEvictions());
+                w.writeLong(metrics.getCacheRemovals());
+                w.writeFloat(metrics.getAveragePutTime());
+                w.writeFloat(metrics.getAverageGetTime());
+                w.writeFloat(metrics.getAverageRemoveTime());
+                w.writeFloat(metrics.getAverageTxCommitTime());
+                w.writeFloat(metrics.getAverageTxRollbackTime());
+                w.writeString(metrics.name());
+                w.writeLong(metrics.getOverflowSize());
+                w.writeLong(metrics.getOffHeapEntriesCount());
+                w.writeLong(metrics.getOffHeapAllocatedSize());
+                w.writeInt(metrics.getSize());
+                w.writeInt(metrics.getKeySize());
+                w.writeBoolean(metrics.isEmpty());
+                w.writeInt(metrics.getDhtEvictQueueCurrentSize());
+                w.writeInt(metrics.getTxThreadMapSize());
+                w.writeInt(metrics.getTxXidMapSize());
+                w.writeInt(metrics.getTxCommitQueueSize());
+                w.writeInt(metrics.getTxPrepareQueueSize());
+                w.writeInt(metrics.getTxStartVersionCountsSize());
+                w.writeInt(metrics.getTxCommittedVersionsSize());
+                w.writeInt(metrics.getTxRolledbackVersionsSize());
+                w.writeInt(metrics.getTxDhtThreadMapSize());
+                w.writeInt(metrics.getTxDhtXidMapSize());
+                w.writeInt(metrics.getTxDhtCommitQueueSize());
+                w.writeInt(metrics.getTxDhtPrepareQueueSize());
+                w.writeInt(metrics.getTxDhtStartVersionCountsSize());
+                w.writeInt(metrics.getTxDhtCommittedVersionsSize());
+                w.writeInt(metrics.getTxDhtRolledbackVersionsSize());
+                w.writeBoolean(metrics.isWriteBehindEnabled());
+                w.writeInt(metrics.getWriteBehindFlushSize());
+                w.writeInt(metrics.getWriteBehindFlushThreadCount());
+                w.writeLong(metrics.getWriteBehindFlushFrequency());
+                w.writeInt(metrics.getWriteBehindStoreBatchSize());
+                w.writeInt(metrics.getWriteBehindTotalCriticalOverflowCount());
+                w.writeInt(metrics.getWriteBehindCriticalOverflowCount());
+                w.writeInt(metrics.getWriteBehindErrorRetryCount());
+                w.writeInt(metrics.getWriteBehindBufferSize());
+                w.writeString(metrics.getKeyType());
+                w.writeString(metrics.getValueType());
+                w.writeBoolean(metrics.isStoreByValue());
+                w.writeBoolean(metrics.isStatisticsEnabled());
+                w.writeBoolean(metrics.isManagementEnabled());
+                w.writeBoolean(metrics.isReadThrough());
+                w.writeBoolean(metrics.isWriteThrough());
+                w.writeFloat(metrics.getCacheHitPercentage());
+                w.writeFloat(metrics.getCacheMissPercentage());
+
+                break;
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET: {
+                writer.writeObjectDetached(cache.get(reader.readObjectDetached()));
+
+                break;
+            }
+
+            case OP_GET_AND_PUT: {
+                writer.writeObjectDetached(cache.getAndPut(reader.readObjectDetached(), reader.readObjectDetached()));
+
+                break;
+            }
+
+            case OP_GET_AND_REPLACE: {
+                writer.writeObjectDetached(cache.getAndReplace(reader.readObjectDetached(),
+                    reader.readObjectDetached()));
+
+                break;
+            }
+
+            case OP_GET_AND_REMOVE: {
+                writer.writeObjectDetached(cache.getAndRemove(reader.readObjectDetached()));
+
+                break;
+            }
+
+            case OP_GET_AND_PUT_IF_ABSENT: {
+                writer.writeObjectDetached(cache.getAndPutIfAbsent(reader.readObjectDetached(), reader.readObjectDetached()));
+
+                break;
+            }
+
+            case OP_PEEK: {
+                Object key = reader.readObjectDetached();
+
+                CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes(reader.readInt());
+
+                writer.writeObjectDetached(cache.localPeek(key, modes));
+
+                break;
+            }
+
+            case OP_GET_ALL: {
+                Set keys = PlatformUtils.readSet(reader);
+
+                Map entries = cache.getAll(keys);
+
+                PlatformUtils.writeNullableMap(writer, entries);
+
+                break;
+            }
+
+            case OP_INVOKE: {
+                Object key = reader.readObjectDetached();
+
+                CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0);
+
+                try {
+                    writer.writeObjectDetached(cache.invoke(key, proc));
+                }
+                catch (EntryProcessorException ex)
+                {
+                    if (ex.getCause() instanceof PlatformNativeException)
+                        writer.writeObjectDetached(((PlatformNativeException)ex.getCause()).cause());
+                    else
+                        throw ex;
+                }
+
+                break;
+            }
+
+            case OP_INVOKE_ALL: {
+                Set<Object> keys = PlatformUtils.readSet(reader);
+
+                CacheEntryProcessor proc = platformCtx.createCacheEntryProcessor(reader.readObjectDetached(), 0);
+
+                writeInvokeAllResult(writer, cache.invokeAll(keys, proc));
+
+                break;
+            }
+
+            case OP_LOCK:
+                writer.writeLong(registerLock(cache.lock(reader.readObjectDetached())));
+
+                break;
+
+            case OP_LOCK_ALL:
+                writer.writeLong(registerLock(cache.lockAll(PlatformUtils.readCollection(reader))));
+
+                break;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Exception convertException(Exception e) {
+        if (e instanceof CachePartialUpdateException)
+            return new PlatformCachePartialUpdateException((CachePartialUpdateException)e, platformCtx, keepPortable);
+
+        return super.convertException(e);
+    }
+
+    /**
+     * Writes the result of InvokeAll cache method.
+     *
+     * @param writer Writer.
+     * @param results Results.
+     */
+    private static void writeInvokeAllResult(PortableRawWriterEx writer, Map<Object, EntryProcessorResult> results) {
+        if (results == null) {
+            writer.writeInt(-1);
+
+            return;
+        }
+
+        writer.writeInt(results.size());
+
+        for (Map.Entry<Object, EntryProcessorResult> entry : results.entrySet()) {
+            writer.writeObjectDetached(entry.getKey());
+
+            EntryProcessorResult procRes = entry.getValue();
+
+            try {
+                Object res = procRes.get();
+
+                writer.writeBoolean(false);  // No exception
+
+                writer.writeObjectDetached(res);
+            }
+            catch (Exception ex) {
+                writer.writeBoolean(true);  // Exception
+
+                writeError(writer, ex);
+            }
+        }
+    }
+
+    /**
+     * Writes an error to the writer either as a native exception, or as a couple of strings.
+     * @param writer Writer.
+     * @param ex Exception.
+     */
+    private static void writeError(PortableRawWriterEx writer, Exception ex) {
+        if (ex.getCause() instanceof PlatformNativeException)
+            writer.writeObjectDetached(((PlatformNativeException)ex.getCause()).cause());
+        else {
+            writer.writeObjectDetached(ex.getClass().getName());
+            writer.writeObjectDetached(ex.getMessage());
+        }
+    }
+
+    /** <inheritDoc /> */
+    @Override protected IgniteFuture currentFuture() throws IgniteCheckedException {
+        return cache.future();
+    }
+
+    /** <inheritDoc /> */
+    @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) {
+        if (opId == OP_GET_ALL)
+            return WRITER_GET_ALL;
+
+        if (opId == OP_INVOKE)
+            return WRITER_PROC_ERR;
+
+        if (opId == OP_INVOKE_ALL)
+            return WRITER_INVOKE_ALL;
+
+        return null;
+    }
+
+    /**
+     * Clears the contents of the cache, without notifying listeners or
+     * {@link javax.cache.integration.CacheWriter}s.
+     *
+     * @throws IllegalStateException if the cache is closed.
+     * @throws javax.cache.CacheException        if there is a problem during the clear
+     */
+    public void clear() throws IgniteCheckedException {
+        cache.clear();
+    }
+
+    /**
+     * Removes all entries.
+     *
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public void removeAll() throws IgniteCheckedException {
+        cache.removeAll();
+    }
+
+    /**
+     * Read cache size.
+     *
+     * @param peekModes Encoded peek modes.
+     * @param loc Local mode flag.
+     * @return Size.
+     */
+    public int size(int peekModes, boolean loc) {
+        CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes(peekModes);
+
+        return loc ? cache.localSize(modes) :  cache.size(modes);
+    }
+
+    /**
+     * Create cache iterator.
+     *
+     * @return Cache iterator.
+     */
+    public PlatformCacheIterator iterator() {
+        Iterator<Cache.Entry> iter = cache.iterator();
+
+        return new PlatformCacheIterator(platformCtx, iter);
+    }
+
+    /**
+     * Create cache iterator over local entries.
+     *
+     * @param peekModes Peke modes.
+     * @return Cache iterator.
+     */
+    public PlatformCacheIterator localIterator(int peekModes) {
+        CachePeekMode[] peekModes0 = PlatformUtils.decodeCachePeekModes(peekModes);
+
+        Iterator<Cache.Entry> iter = cache.localEntries(peekModes0).iterator();
+
+        return new PlatformCacheIterator(platformCtx, iter);
+    }
+
+    /**
+     * Enters a lock.
+     *
+     * @param id Lock id.
+     */
+    public void enterLock(long id) throws InterruptedException {
+        lock(id).lockInterruptibly();
+    }
+
+    /**
+     * Exits a lock.
+     *
+     * @param id Lock id.
+     */
+    public void exitLock(long id) {
+        lock(id).unlock();
+    }
+
+    /**
+     * Attempts to enter a lock.
+     *
+     * @param id Lock id.
+     * @param timeout Timeout, in milliseconds. -1 for infinite timeout.
+     */
+    public boolean tryEnterLock(long id, long timeout) throws InterruptedException {
+        return timeout == -1
+            ? lock(id).tryLock()
+            : lock(id).tryLock(timeout, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Rebalances the cache.
+     *
+     * @param futId Future id.
+     */
+    public void rebalance(long futId) {
+        PlatformFutureUtils.listen(platformCtx, cache.rebalance().chain(new C1<IgniteFuture, Object>() {
+            @Override
+            public Object apply(IgniteFuture fut) {
+                return null;
+            }
+        }), futId, PlatformFutureUtils.TYP_OBJ);
+    }
+
+    /**
+     * Unregister lock.
+     *
+     * @param id Lock id.
+     */
+    public void closeLock(long id){
+        Lock lock = lockMap.remove(id);
+
+        assert lock != null : "Failed to unregister lock: " + id;
+    }
+
+    /**
+     * Get lock by id.
+     *
+     * @param id Id.
+     * @return Lock.
+     */
+    private Lock lock(long id) {
+        Lock lock = lockMap.get(id);
+
+        assert lock != null : "Lock not found for ID: " + id;
+
+        return lock;
+    }
+
+    /**
+     * Registers a lock in a map.
+     *
+     * @param lock Lock to register.
+     * @return Registered lock id.
+     */
+    private long registerLock(Lock lock) {
+        long id = LOCK_ID_GEN.incrementAndGet();
+
+        lockMap.put(id, lock);
+
+        return id;
+    }
+
+    /**
+     * Runs specified query.
+     */
+    private PlatformQueryCursor runQuery(PortableRawReaderEx reader, Query qry)
+        throws IgniteCheckedException {
+
+        try {
+            QueryCursorEx cursor = (QueryCursorEx) cache.query(qry);
+
+            return new PlatformQueryCursor(platformCtx, cursor,
+                qry.getPageSize() > 0 ? qry.getPageSize(): Query.DFLT_PAGE_SIZE);
+        }
+        catch (Exception err) {
+            throw PlatformUtils.unwrapQueryException(err);
+        }
+    }
+
+    /**
+     * Runs specified fields query.
+     */
+    private PlatformFieldsQueryCursor runFieldsQuery(PortableRawReaderEx reader, Query qry)
+        throws IgniteCheckedException {
+        try {
+            QueryCursorEx cursor = (QueryCursorEx) cache.query(qry);
+
+            return new PlatformFieldsQueryCursor(platformCtx, cursor,
+                qry.getPageSize() > 0 ? qry.getPageSize() : Query.DFLT_PAGE_SIZE);
+        }
+        catch (Exception err) {
+            throw PlatformUtils.unwrapQueryException(err);
+        }
+    }
+
+    /**
+     * Reads the query of specified type.
+     */
+    private Query readInitialQuery(PortableRawReaderEx reader)
+        throws IgniteCheckedException {
+        int typ = reader.readInt();
+
+        switch (typ) {
+            case -1:
+                return null;
+
+            case OP_QRY_SCAN:
+                return readScanQuery(reader);
+
+            case OP_QRY_SQL:
+                return readSqlQuery(reader);
+
+            case OP_QRY_TXT:
+                return readTextQuery(reader);
+        }
+
+        throw new IgniteCheckedException("Unsupported query type: " + typ);
+    }
+
+    /**
+     * Reads sql query.
+     */
+    private Query readSqlQuery(PortableRawReaderEx reader) {
+        boolean loc = reader.readBoolean();
+        String sql = reader.readString();
+        String typ = reader.readString();
+        final int pageSize = reader.readInt();
+
+        Object[] args = readQueryArgs(reader);
+
+        return new SqlQuery(typ, sql).setPageSize(pageSize).setArgs(args).setLocal(loc);
+    }
+
+    /**
+     * Reads fields query.
+     */
+    private Query readFieldsQuery(PortableRawReaderEx reader) {
+        boolean loc = reader.readBoolean();
+        String sql = reader.readString();
+        final int pageSize = reader.readInt();
+
+        Object[] args = readQueryArgs(reader);
+
+        return new SqlFieldsQuery(sql).setPageSize(pageSize).setArgs(args).setLocal(loc);
+    }
+
+    /**
+     * Reads text query.
+     */
+    private Query readTextQuery(PortableRawReaderEx reader) {
+        boolean loc = reader.readBoolean();
+        String txt = reader.readString();
+        String typ = reader.readString();
+        final int pageSize = reader.readInt();
+
+        return new TextQuery(typ, txt).setPageSize(pageSize).setLocal(loc);
+    }
+
+    /**
+     * Reads scan query.
+     */
+    private Query readScanQuery(PortableRawReaderEx reader) {
+        boolean loc = reader.readBoolean();
+        final int pageSize = reader.readInt();
+
+        boolean hasPart = reader.readBoolean();
+
+        Integer part = hasPart ? reader.readInt() : null;
+
+        ScanQuery qry = new ScanQuery().setPageSize(pageSize);
+
+        qry.setPartition(part);
+
+        Object pred = reader.readObjectDetached();
+
+        if (pred != null)
+            qry.setFilter(platformCtx.createCacheEntryFilter(pred, reader.readLong()));
+
+        qry.setLocal(loc);
+
+        return qry;
+    }
+
+    /**
+     * Writes error with EntryProcessorException cause.
+     */
+    private static class GetAllWriter implements PlatformFutureUtils.Writer {
+        /** <inheritDoc /> */
+        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+            assert obj instanceof Map;
+
+            PlatformUtils.writeNullableMap(writer, (Map) obj);
+        }
+
+        /** <inheritDoc /> */
+        @Override public boolean canWrite(Object obj, Throwable err) {
+            return err == null;
+        }
+    }
+
+    /**
+     * Writes error with EntryProcessorException cause.
+     */
+    private static class EntryProcessorExceptionWriter implements PlatformFutureUtils.Writer {
+        /** <inheritDoc /> */
+        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+            EntryProcessorException entryEx = (EntryProcessorException) err;
+
+            writeError(writer, entryEx);
+        }
+
+        /** <inheritDoc /> */
+        @Override public boolean canWrite(Object obj, Throwable err) {
+            return err instanceof EntryProcessorException;
+        }
+    }
+
+    /**
+     * Writes results of InvokeAll method.
+     */
+    private static class EntryProcessorResultsWriter implements PlatformFutureUtils.Writer {
+        /** <inheritDoc /> */
+        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+            writeInvokeAllResult(writer, (Map)obj);
+        }
+
+        /** <inheritDoc /> */
+        @Override public boolean canWrite(Object obj, Throwable err) {
+            return obj != null && err == null;
+        }
+    }
+
+    /**
+     * Interop expiry policy.
+     */
+    private static class InteropExpiryPolicy implements ExpiryPolicy {
+        /** Duration: unchanged. */
+        private static final long DUR_UNCHANGED = -2;
+
+        /** Duration: eternal. */
+        private static final long DUR_ETERNAL = -1;
+
+        /** Duration: zero. */
+        private static final long DUR_ZERO = 0;
+
+        /** Expiry for create. */
+        private final Duration create;
+
+        /** Expiry for update. */
+        private final Duration update;
+
+        /** Expiry for access. */
+        private final Duration access;
+
+        /**
+         * Constructor.
+         *
+         * @param create Expiry for create.
+         * @param update Expiry for update.
+         * @param access Expiry for access.
+         */
+        public InteropExpiryPolicy(long create, long update, long access) {
+            this.create = convert(create);
+            this.update = convert(update);
+            this.access = convert(access);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Duration getExpiryForCreation() {
+            return create;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Duration getExpiryForUpdate() {
+            return update;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Duration getExpiryForAccess() {
+            return access;
+        }
+
+        /**
+         * Convert encoded duration to actual duration.
+         *
+         * @param dur Encoded duration.
+         * @return Actual duration.
+         */
+        private static Duration convert(long dur) {
+            if (dur == DUR_UNCHANGED)
+                return null;
+            else if (dur == DUR_ETERNAL)
+                return Duration.ETERNAL;
+            else if (dur == DUR_ZERO)
+                return Duration.ZERO;
+            else {
+                assert dur > 0;
+
+                return new Duration(TimeUnit.MILLISECONDS, dur);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/207b6820/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
new file mode 100644
index 0000000..fee2995
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
@@ -0,0 +1,105 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.resources.*;
+
+/**
+ * Interop filter. Delegates apply to native platform.
+ */
+public class PlatformCacheEntryFilterImpl<K, V> extends PlatformAbstractPredicate
+    implements PlatformCacheEntryFilter<K, V> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    public PlatformCacheEntryFilterImpl() {
+        super();
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param pred .Net portable predicate.
+     * @param ptr Pointer to predicate in the native platform.
+     * @param ctx Kernal context.
+     */
+    public PlatformCacheEntryFilterImpl(Object pred, long ptr, PlatformContext ctx) {
+        super(pred, ptr, ctx);
+
+        assert pred != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(K k, V v) {
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObject(k);
+            writer.writeObject(v);
+
+            out.synchronize();
+
+            return ctx.gateway().cacheEntryFilterApply(ptr, mem.pointer()) != 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        if (ptr == 0)
+            return;
+
+        assert ctx != null;
+
+        ctx.gateway().cacheEntryFilterDestroy(ptr);
+
+        ptr = 0;
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     */
+    @IgniteInstanceResource
+    public void setIgniteInstance(Ignite ignite) {
+        ctx = PlatformUtils.platformContext(ignite);
+
+        if (ptr != 0)
+            return;
+
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = ctx.writer(out);
+
+            writer.writeObject(pred);
+
+            out.synchronize();
+
+            ptr = ctx.gateway().cacheEntryFilterCreate(mem.pointer());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/207b6820/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
new file mode 100644
index 0000000..ab9ad7c
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessor.java
@@ -0,0 +1,212 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import javax.cache.processor.*;
+import java.io.*;
+
+/**
+ * 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();
+    }
+}


[15/50] [abbrv] 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/c8bc1f92
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/c8bc1f92
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/c8bc1f92

Branch: refs/heads/ignite-843
Commit: c8bc1f92475f7ea4b216de7b024ce4e643db26e3
Parents: fc60fee e4ba2eb
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 12:08:17 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 12:08:17 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      |  3 --
 .../processors/platform/PlatformContext.java    |  9 ++++
 .../platform/utils/PlatformUtils.java           | 55 ++++++++++++++++++++
 3 files changed, 64 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[48/50] [abbrv] ignite git commit: ignite-1304 Changed GridNearOptimisticTxPrepareFuture to do not execute whole prepare process with topology read lock held

Posted by ak...@apache.org.
ignite-1304 Changed GridNearOptimisticTxPrepareFuture to do not execute whole prepare process with topology read lock held


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

Branch: refs/heads/ignite-843
Commit: 37a0505c11136321e91caa6db41e74c6ef89b734
Parents: e2f522b
Author: sboikov <sb...@gridgain.com>
Authored: Fri Aug 28 16:57:23 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Aug 28 16:57:23 2015 +0300

----------------------------------------------------------------------
 .../near/GridNearOptimisticTxPrepareFuture.java | 87 ++++++++++---------
 ...acheAsyncOperationsFailoverAbstractTest.java | 91 +++++++++++++-------
 2 files changed, 110 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/37a0505c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 8e66cb6..2d6b2a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -227,6 +227,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
         if (topVer != null) {
             tx.topologyVersion(topVer);
 
+            cctx.mvcc().addFuture(this);
+
             prepare0(false);
 
             return;
@@ -242,6 +244,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
     private void prepareOnTopology(final boolean remap, @Nullable final Runnable c) {
         GridDhtTopologyFuture topFut = topologyReadLock();
 
+        AffinityTopologyVersion topVer = null;
+
         try {
             if (topFut == null) {
                 assert isDone();
@@ -250,52 +254,61 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             }
 
             if (topFut.isDone()) {
-                StringBuilder invalidCaches = new StringBuilder();
+                topVer = topFut.topologyVersion();
 
-                boolean cacheInvalid = false;
+                if (remap)
+                    tx.onRemap(topVer);
+                else
+                    tx.topologyVersion(topVer);
 
-                for (GridCacheContext ctx : cctx.cacheContexts()) {
-                    if (tx.activeCacheIds().contains(ctx.cacheId()) && !topFut.isCacheTopologyValid(ctx)) {
-                        if (cacheInvalid)
-                            invalidCaches.append(", ");
+                if (!remap)
+                    cctx.mvcc().addFuture(this);
+            }
+        }
+        finally {
+            topologyReadUnlock();
+        }
 
-                        invalidCaches.append(U.maskName(ctx.name()));
+        if (topVer != null) {
+            StringBuilder invalidCaches = null;
 
-                        cacheInvalid = true;
-                    }
-                }
+            for (Integer cacheId : tx.activeCacheIds()) {
+                GridCacheContext ctx = cctx.cacheContext(cacheId);
 
-                if (cacheInvalid) {
-                    onDone(new IgniteCheckedException("Failed to perform cache operation (cache topology is not valid): " +
-                        invalidCaches.toString()));
+                assert ctx != null : cacheId;
 
-                    return;
-                }
+                if (!topFut.isCacheTopologyValid(ctx)) {
+                    if (invalidCaches != null)
+                        invalidCaches.append(", ");
+                    else
+                        invalidCaches = new StringBuilder();
 
-                if (remap)
-                    tx.onRemap(topFut.topologyVersion());
-                else
-                    tx.topologyVersion(topFut.topologyVersion());
+                    invalidCaches.append(U.maskName(ctx.name()));
+                }
+            }
 
-                prepare0(remap);
+            if (invalidCaches != null) {
+                onDone(new IgniteCheckedException("Failed to perform cache operation (cache topology is not valid): " +
+                    invalidCaches.toString()));
 
-                if (c != null)
-                    c.run();
-            }
-            else {
-                topFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                        cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
-                            @Override public void run() {
-                                prepareOnTopology(remap, c);
-                            }
-                        });
-                    }
-                });
+                return;
             }
+
+            prepare0(remap);
+
+            if (c != null)
+                c.run();
         }
-        finally {
-            topologyReadUnlock();
+        else {
+            topFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                    cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
+                        @Override public void run() {
+                            prepareOnTopology(remap, c);
+                        }
+                    });
+                }
+            });
         }
     }
 
@@ -382,10 +395,6 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                 return;
             }
 
-            // Make sure to add future before calling prepare.
-            if (!remap)
-                cctx.mvcc().addFuture(this);
-
             prepare(
                 tx.optimistic() && tx.serializable() ? tx.readEntries() : Collections.<IgniteTxEntry>emptyList(),
                 tx.writeEntries());

http://git-wip-us.apache.org/repos/asf/ignite/blob/37a0505c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
index 1669404..36eb9a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
@@ -60,6 +60,7 @@ public abstract class CacheAsyncOperationsFailoverAbstractTest extends GridCache
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
         CacheConfiguration ccfg = super.cacheConfiguration(gridName);
 
@@ -93,59 +94,91 @@ public abstract class CacheAsyncOperationsFailoverAbstractTest extends GridCache
      * @throws Exception If failed.
      */
     public void testAsyncFailover() throws Exception {
-        for (int i = 0; i < 3; i++) {
+        IgniteCache<TestKey, TestValue> cache = ignite(0).<TestKey, TestValue>cache(null).withAsync();
+
+        int ops = cache.getConfiguration(CacheConfiguration.class).getMaxConcurrentAsyncOperations();
+
+        log.info("Max concurrent async operations: " + ops);
+
+        assertTrue(ops > 0);
+
+        // Start/stop one node.
+        for (int i = 0; i < 2; i++) {
             log.info("Iteration: " + i);
 
             startGrid(NODE_CNT);
 
-            final IgniteCache<TestKey, TestValue> cache = ignite(0).<TestKey, TestValue>cache(null).withAsync();
+            List<IgniteFuture<?>> futs = startAsyncOperations(ops, cache);
 
-            int ops = cache.getConfiguration(CacheConfiguration.class).getMaxConcurrentAsyncOperations();
+            stopGrid(NODE_CNT);
 
-            log.info("Max concurrent async operations: " + ops);
+            for (IgniteFuture<?> fut : futs)
+                fut.get();
 
-            assertTrue(ops > 0);
+            log.info("Iteration done: " + i);
+        }
 
-            final List<IgniteFuture<?>> futs = Collections.synchronizedList(new ArrayList<IgniteFuture<?>>(ops));
+        // Start all nodes except one.
+        try {
+            List<IgniteFuture<?>> futs = startAsyncOperations(ops, cache);
 
-            final AtomicInteger left = new AtomicInteger(ops);
+            for (int i = 1; i < NODE_CNT; i++)
+                stopGrid(i);
 
-            GridTestUtils.runMultiThreaded(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    List<IgniteFuture<?>> futs0 = new ArrayList<>();
+            for (IgniteFuture<?> fut : futs)
+                fut.get();
+        }
+        finally {
+            for (int i = 1; i < NODE_CNT; i++)
+                startGrid(i);
+        }
+    }
 
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+    /**
+     * @param ops Number of operations.
+     * @param cache Cache.
+     * @return Futures.
+     * @throws Exception If failed.
+     */
+    private List<IgniteFuture<?>> startAsyncOperations(final int ops, final IgniteCache<TestKey, TestValue> cache)
+        throws Exception
+    {
+        final List<IgniteFuture<?>> futs = Collections.synchronizedList(new ArrayList<IgniteFuture<?>>(ops));
 
-                    while (left.getAndDecrement() > 0) {
-                        TreeMap<TestKey, TestValue> map = new TreeMap<>();
+        final AtomicInteger left = new AtomicInteger(ops);
 
-                        int keys = 50;
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                List<IgniteFuture<?>> futs0 = new ArrayList<>();
 
-                        for (int k = 0; k < keys; k++)
-                            map.put(new TestKey(rnd.nextInt(10_000)), new TestValue(k));
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
-                        cache.putAll(map);
+                while (left.getAndDecrement() > 0) {
+                    TreeMap<TestKey, TestValue> map = new TreeMap<>();
 
-                        IgniteFuture<?> fut = cache.future();
+                    int keys = 50;
 
-                        assertNotNull(fut);
+                    for (int k = 0; k < keys; k++)
+                        map.put(new TestKey(rnd.nextInt(10_000)), new TestValue(k));
 
-                        futs0.add(fut);
-                    }
+                    cache.putAll(map);
 
-                    futs.addAll(futs0);
+                    IgniteFuture<?> fut = cache.future();
 
-                    return null;
+                    assertNotNull(fut);
+
+                    futs0.add(fut);
                 }
-            }, 10, "put-thread");
 
-            stopGrid(NODE_CNT);
+                futs.addAll(futs0);
+
+                return null;
+            }
+        }, 10, "put-thread");
 
-            assertEquals(ops, futs.size());
+        assertEquals(ops, futs.size());
 
-            for (IgniteFuture<?> fut : futs)
-                fut.get();
-        }
+        return futs;
     }
 
     /**


[11/50] [abbrv] ignite git commit: Further refactorings necessary for platforms move to Ignite.

Posted by ak...@apache.org.
Further refactorings necessary for platforms move to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 5877b301a89ed87531d503fe92474d8a8568134a
Parents: 136c099
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 09:53:17 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 09:53:17 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    | 114 +++
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../callback/PlatformCallbackGateway.java       | 869 +++++++++++++++++++
 .../callback/PlatformCallbackUtils.java         | 468 ++++++++++
 .../platform/memory/PlatformInputStream.java    |  30 +
 .../platform/memory/PlatformMemory.java         |  77 ++
 .../platform/memory/PlatformMemoryManager.java  |  46 +
 .../platform/memory/PlatformOutputStream.java   |  30 +
 .../processors/platform/PlatformContext.java    | 114 ---
 .../callback/PlatformCallbackGateway.java       | 869 -------------------
 .../callback/PlatformCallbackUtils.java         | 468 ----------
 .../platform/memory/PlatformInputStream.java    |  30 -
 .../platform/memory/PlatformMemory.java         |  77 --
 .../platform/memory/PlatformMemoryManager.java  |  46 -
 .../platform/memory/PlatformOutputStream.java   |  30 -
 .../platform/utils/PlatformUtils.java           |  23 +
 17 files changed, 1669 insertions(+), 1634 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/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
new file mode 100644
index 0000000..fb1eaa2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.callback.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+
+import java.util.*;
+
+/**
+ * Platform context. Acts as an entry point for platform operations.
+ */
+public interface PlatformContext {
+    /**
+     * Gets kernal context.
+     *
+     * @return Kernal context.
+     */
+    public GridKernalContext kernalContext();
+
+    /**
+     * Gets platform memory manager.
+     *
+     * @return Memory manager.
+     */
+    public PlatformMemoryManager memory();
+
+    /**
+     * Gets platform callback gateway.
+     *
+     * @return Callback gateway.
+     */
+    public PlatformCallbackGateway gateway();
+
+    /**
+     * Get memory reader.
+     *
+     * @param mem Memory.
+     * @return Reader.
+     */
+    public PortableRawReaderEx reader(PlatformMemory mem);
+
+    /**
+     * Get memory reader.
+     *
+     * @param in Input.
+     * @return Reader.
+     */
+    public PortableRawReaderEx reader(PlatformInputStream in);
+
+    /**
+     * Get memory writer.
+     *
+     * @param mem Memory.
+     * @return Writer.
+     */
+    public PortableRawWriterEx writer(PlatformMemory mem);
+
+    /**
+     * Get memory writer.
+     *
+     * @param out Output.
+     * @return Writer.
+     */
+    public PortableRawWriterEx writer(PlatformOutputStream out);
+
+    /**
+     * Sends node info to native platform, if necessary.
+     *
+     * @param node Node.
+     */
+    public void addNode(ClusterNode node);
+
+    /**
+     * Writes a node id to a stream and sends node info to native platform, if necessary.
+     *
+     * @param writer Writer.
+     * @param node Node.
+     */
+    public void writeNode(PortableRawWriterEx writer, ClusterNode node);
+
+    /**
+     * Writes multiple node ids to a stream and sends node info to native platform, if necessary.
+     *
+     * @param writer Writer.
+     * @param nodes Nodes.
+     */
+    public void writeNodes(PortableRawWriterEx writer, Collection<ClusterNode> nodes);
+
+    /**
+     * Process metadata from the platform.
+     *
+     * @param reader Reader.
+     */
+    public void processMetadata(PortableRawReaderEx reader);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index 245d4d7..9bdc3be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -38,4 +38,9 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public long environmentPointer() {
         return 0;
     }
+
+    /** {@inheritDoc} */
+    @Override public PlatformContext context() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 137c31b..782db4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -37,4 +37,11 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Environment pointer.
      */
     public long environmentPointer();
+
+    /**
+     * Gets platform context.
+     *
+     * @return Platform context.
+     */
+    public PlatformContext context();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/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
new file mode 100644
index 0000000..a8e7879
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -0,0 +1,869 @@
+/*
+ * 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.callback;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.*;
+
+/**
+ * Gateway to all platform-dependent callbacks. Implementers might extend this class and provide additional callbacks.
+ */
+@SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"})
+public class PlatformCallbackGateway {
+    /** Environment pointer. */
+    protected final long envPtr;
+
+    /** Lock. */
+    private final GridSpinReadWriteLock lock = new GridSpinReadWriteLock();
+
+    /**
+     * Native gateway.
+     *
+     * @param envPtr Environment pointer.
+     */
+    public PlatformCallbackGateway(long envPtr) {
+        this.envPtr = envPtr;
+    }
+
+    /**
+     * Get environment pointer.
+     *
+     * @return Environment pointer.
+     */
+    public long environmentPointer() {
+        return envPtr;
+    }
+
+    /**
+     * Create cache store.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    public long cacheStoreCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.cacheStoreCreate(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param objPtr Object pointer.
+     * @param memPtr Memory pointer.
+     * @param cb Callback.
+     * @return Result.
+     */
+    public int cacheStoreInvoke(long objPtr, long memPtr, Object cb) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.cacheStoreInvoke(envPtr, objPtr, memPtr, cb);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param objPtr Object pointer.
+     */
+    public void cacheStoreDestroy(long objPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.cacheStoreDestroy(envPtr, objPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Creates cache store session.
+     *
+     * @param storePtr Store instance pointer.
+     * @return Session instance pointer.
+     */
+    public long cacheStoreSessionCreate(long storePtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.cacheStoreSessionCreate(envPtr, storePtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Creates cache entry filter and returns a pointer.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    public long cacheEntryFilterCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.cacheEntryFilterCreate(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    public int cacheEntryFilterApply(long ptr, long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.cacheEntryFilterApply(envPtr, ptr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param ptr Pointer.
+     */
+    public void cacheEntryFilterDestroy(long ptr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.cacheEntryFilterDestroy(envPtr, ptr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Invoke cache entry processor.
+     *
+     * @param outMemPtr Output memory pointer.
+     * @param inMemPtr Input memory pointer.
+     */
+    public void cacheInvoke(long outMemPtr, long inMemPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.cacheInvoke(envPtr, outMemPtr, inMemPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Perform native task map. Do not throw exceptions, serializing them to the output stream instead.
+     *
+     * @param taskPtr Task pointer.
+     * @param outMemPtr Output memory pointer (exists if topology changed, otherwise {@code 0}).
+     * @param inMemPtr Input memory pointer.
+     */
+    public void computeTaskMap(long taskPtr, long outMemPtr, long inMemPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.computeTaskMap(envPtr, taskPtr, outMemPtr, inMemPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Perform native task job result notification.
+     *
+     * @param taskPtr Task pointer.
+     * @param jobPtr Job pointer.
+     * @param memPtr Memory pointer (always zero for local job execution).
+     * @return Job result enum ordinal.
+     */
+    public int computeTaskJobResult(long taskPtr, long jobPtr, long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.computeTaskJobResult(envPtr, taskPtr, jobPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Perform native task reduce.
+     *
+     * @param taskPtr Task pointer.
+     */
+    public void computeTaskReduce(long taskPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.computeTaskReduce(envPtr, taskPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Complete task with native error.
+     *
+     * @param taskPtr Task pointer.
+     * @param memPtr Memory pointer with exception data or {@code 0} in case of success.
+     */
+    public void computeTaskComplete(long taskPtr, long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.computeTaskComplete(envPtr, taskPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Serialize native job.
+     *
+     * @param jobPtr Job pointer.
+     * @param memPtr Memory pointer.
+     * @return {@code True} if serialization succeeded.
+     */
+    public int computeJobSerialize(long jobPtr, long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.computeJobSerialize(envPtr, jobPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Create job in native platform.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer to job.
+     */
+    public long computeJobCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.computeJobCreate(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Execute native job on a node other than where it was created.
+     *
+     * @param jobPtr Job pointer.
+     * @param cancel Cancel flag.
+     * @param memPtr Memory pointer to write result to for remote job execution or {@code 0} for local job execution.
+     */
+    public void computeJobExecute(long jobPtr, int cancel, long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.computeJobExecute(envPtr, jobPtr, cancel, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Cancel the job.
+     *
+     * @param jobPtr Job pointer.
+     */
+    public void computeJobCancel(long jobPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.computeJobCancel(envPtr, jobPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Destroy the job.
+     *
+     * @param ptr Pointer.
+     */
+    public void computeJobDestroy(long ptr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.computeJobDestroy(envPtr, ptr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Invoke local callback.
+     *
+     * @param cbPtr Callback pointer.
+     * @param memPtr Memory pointer.
+     */
+    public void continuousQueryListenerApply(long cbPtr, long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.continuousQueryListenerApply(envPtr, cbPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Create filter in native platform.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer to created filter.
+     */
+    public long continuousQueryFilterCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.continuousQueryFilterCreate(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Invoke remote filter.
+     *
+     * @param filterPtr Filter pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    public int continuousQueryFilterApply(long filterPtr, long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.continuousQueryFilterApply(envPtr, filterPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Release remote  filter.
+     *
+     * @param filterPtr Filter pointer.
+     */
+    public void continuousQueryFilterRelease(long filterPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.continuousQueryFilterRelease(envPtr, filterPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify native data streamer about topology update.
+     *
+     * @param ptr Data streamer native pointer.
+     * @param topVer Topology version.
+     * @param topSize Topology size.
+     */
+    public void dataStreamerTopologyUpdate(long ptr, long topVer, int topSize) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.dataStreamerTopologyUpdate(envPtr, ptr, topVer, topSize);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Invoke stream receiver.
+     *
+     * @param ptr Receiver native pointer.
+     * @param cache Cache object.
+     * @param memPtr Stream pointer.
+     * @param keepPortable Portable flag.
+     */
+    public void dataStreamerStreamReceiverInvoke(long ptr, Object cache, long memPtr, boolean keepPortable) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.dataStreamerStreamReceiverInvoke(envPtr, ptr, cache, memPtr, keepPortable);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with byte result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureByteResult(long futPtr, int res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureByteResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with boolean result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureBoolResult(long futPtr, int res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureBoolResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with short result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureShortResult(long futPtr, int res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureShortResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with byte result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureCharResult(long futPtr, int res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureCharResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with int result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureIntResult(long futPtr, int res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureIntResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with float result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureFloatResult(long futPtr, float res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureFloatResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with long result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureLongResult(long futPtr, long res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureLongResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with double result.
+     *
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    public void futureDoubleResult(long futPtr, double res) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureDoubleResult(envPtr, futPtr, res);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with object result.
+     *
+     * @param futPtr Future pointer.
+     * @param memPtr Memory pointer.
+     */
+    public void futureObjectResult(long futPtr, long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureObjectResult(envPtr, futPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with null result.
+     *
+     * @param futPtr Future pointer.
+     */
+    public void futureNullResult(long futPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureNullResult(envPtr, futPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify future with error.
+     *
+     * @param futPtr Future pointer.
+     * @param memPtr Pointer to memory with error information.
+     */
+    public void futureError(long futPtr, long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.futureError(envPtr, futPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Creates message filter and returns a pointer.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    public long messagingFilterCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.messagingFilterCreate(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    public int messagingFilterApply(long ptr, long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.messagingFilterApply(envPtr, ptr, memPtr);
+        }
+        finally {
+            leave();
+        }}
+
+    /**
+     * @param ptr Pointer.
+     */
+    public void messagingFilterDestroy(long ptr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.messagingFilterDestroy(envPtr, ptr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Creates event filter and returns a pointer.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    public long eventFilterCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.eventFilterCreate(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    public int eventFilterApply(long ptr, long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.eventFilterApply(envPtr, ptr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * @param ptr Pointer.
+     */
+    public void eventFilterDestroy(long ptr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.eventFilterDestroy(envPtr, ptr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Sends node info to native target.
+     *
+     * @param memPtr Ptr to a stream with serialized node.
+     */
+    public void nodeInfo(long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.nodeInfo(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Kernal start callback.
+     *
+     * @param memPtr Memory pointer.
+     */
+    public void onStart(long memPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.onStart(envPtr, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Lifecycle event callback.
+     *
+     * @param ptr Holder pointer.
+     * @param evt Event.
+     */
+    public void lifecycleEvent(long ptr, int evt) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.lifecycleEvent(envPtr, ptr, evt);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Re-allocate external memory chunk.
+     *
+     * @param memPtr Cross-platform pointer.
+     * @param cap Capacity.
+     */
+    public void memoryReallocate(long memPtr, int cap) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.memoryReallocate(envPtr, memPtr, cap);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Initializes native service.
+     *
+     * @param memPtr Pointer.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public long serviceInit(long memPtr) throws IgniteCheckedException {
+        return PlatformCallbackUtils.serviceInit(envPtr, memPtr);
+    }
+
+    /**
+     * Executes native service.
+     *
+     * @param svcPtr Pointer to the service in the native platform.
+     * @param memPtr Stream pointer.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public void serviceExecute(long svcPtr, long memPtr) throws IgniteCheckedException {
+        PlatformCallbackUtils.serviceExecute(envPtr, svcPtr, memPtr);
+    }
+
+    /**
+     * Cancels native service.
+     *
+     * @param svcPtr Pointer to the service in the native platform.
+     * @param memPtr Stream pointer.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public void serviceCancel(long svcPtr, long memPtr) throws IgniteCheckedException {
+        PlatformCallbackUtils.serviceCancel(envPtr, svcPtr, memPtr);
+    }
+
+    /**
+     * Invokes service method.
+     *
+     * @param svcPtr Pointer to the service in the native platform.
+     * @param outMemPtr Output memory pointer.
+     * @param inMemPtr Input memory pointer.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    public void serviceInvokeMethod(long svcPtr, long outMemPtr, long inMemPtr) throws IgniteCheckedException {
+        PlatformCallbackUtils.serviceInvokeMethod(envPtr, svcPtr, outMemPtr, inMemPtr);
+    }
+
+    /**
+     * Invokes cluster node filter.
+     *
+     * @param memPtr Stream pointer.
+     */
+    public int clusterNodeFilterApply(long memPtr) {
+        return PlatformCallbackUtils.clusterNodeFilterApply(envPtr, memPtr);
+    }
+
+    /**
+     * Kernal stop callback.
+     */
+    public void onStop() {
+        block();
+
+        PlatformCallbackUtils.onStop(envPtr);
+    }
+
+    /**
+     * Enter gateway.
+     */
+    protected void enter() {
+        if (!lock.tryReadLock())
+            throw new IgniteException("Failed to execute native callback because grid is stopping.");
+    }
+
+    /**
+     * Leave gateway.
+     */
+    protected void leave() {
+        lock.readUnlock();
+    }
+
+    /**
+     * Block gateway.
+     */
+    protected void block() {
+        lock.writeLock();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
new file mode 100644
index 0000000..7e9587f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
@@ -0,0 +1,468 @@
+/*
+ * 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.callback;
+
+/**
+ * Platform callback utility methods. Implemented in target platform. All methods in this class must be
+ * package-visible and invoked only through {@link PlatformCallbackGateway}.
+ */
+public class PlatformCallbackUtils {
+    /**
+     * Create cache store.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    static native long cacheStoreCreate(long envPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Object pointer.
+     * @param memPtr Memory pointer.
+     * @param cb Callback.
+     * @return Result.
+     */
+    static native int cacheStoreInvoke(long envPtr, long objPtr, long memPtr, Object cb);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Object pointer.
+     */
+    static native void cacheStoreDestroy(long envPtr, long objPtr);
+
+    /**
+     * Creates cache store session.
+     *
+     * @param envPtr Environment pointer.
+     * @param storePtr Store instance pointer.
+     * @return Session instance pointer.
+     */
+    static native long cacheStoreSessionCreate(long envPtr, long storePtr);
+
+    /**
+     * Creates cache entry filter and returns a pointer.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    static native long cacheEntryFilterCreate(long envPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    static native int cacheEntryFilterApply(long envPtr, long objPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Pointer.
+     */
+    static native void cacheEntryFilterDestroy(long envPtr, long objPtr);
+
+    /**
+     * Invoke cache entry processor.
+     *
+     * @param envPtr Environment pointer.
+     * @param outMemPtr Output memory pointer.
+     * @param inMemPtr Input memory pointer.
+     */
+    static native void cacheInvoke(long envPtr, long outMemPtr, long inMemPtr);
+
+    /**
+     * Perform native task map. Do not throw exceptions, serializing them to the output stream instead.
+     *
+     * @param envPtr Environment pointer.
+     * @param taskPtr Task pointer.
+     * @param outMemPtr Output memory pointer (exists if topology changed, otherwise {@code 0}).
+     * @param inMemPtr Input memory pointer.
+     */
+    static native void computeTaskMap(long envPtr, long taskPtr, long outMemPtr, long inMemPtr);
+
+    /**
+     * Perform native task job result notification.
+     *
+     * @param envPtr Environment pointer.
+     * @param taskPtr Task pointer.
+     * @param jobPtr Job pointer.
+     * @param memPtr Memory pointer (always zero for local job execution).
+     * @return Job result enum ordinal.
+     */
+    static native int computeTaskJobResult(long envPtr, long taskPtr, long jobPtr, long memPtr);
+
+    /**
+     * Perform native task reduce.
+     *
+     * @param envPtr Environment pointer.
+     * @param taskPtr Task pointer.
+     */
+    static native void computeTaskReduce(long envPtr, long taskPtr);
+
+    /**
+     * Complete task with native error.
+     *
+     * @param envPtr Environment pointer.
+     * @param taskPtr Task pointer.
+     * @param memPtr Memory pointer with exception data or {@code 0} in case of success.
+     */
+    static native void computeTaskComplete(long envPtr, long taskPtr, long memPtr);
+
+    /**
+     * Serialize native job.
+     *
+     * @param envPtr Environment pointer.
+     * @param jobPtr Job pointer.
+     * @param memPtr Memory pointer.
+     * @return {@code True} if serialization succeeded.
+     */
+    static native int computeJobSerialize(long envPtr, long jobPtr, long memPtr);
+
+    /**
+     * Create job in native platform.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     * @return Pointer to job.
+     */
+    static native long computeJobCreate(long envPtr, long memPtr);
+
+    /**
+     * Execute native job on a node other than where it was created.
+     *
+     * @param envPtr Environment pointer.
+     * @param jobPtr Job pointer.
+     * @param cancel Cancel flag.
+     * @param memPtr Memory pointer to write result to for remote job execution or {@code 0} for local job execution.
+     */
+    static native void computeJobExecute(long envPtr, long jobPtr, int cancel, long memPtr);
+
+    /**
+     * Cancel the job.
+     *
+     * @param envPtr Environment pointer.
+     * @param jobPtr Job pointer.
+     */
+    static native void computeJobCancel(long envPtr, long jobPtr);
+
+    /**
+     * Destroy the job.
+     *
+     * @param envPtr Environment pointer.
+     * @param ptr Pointer.
+     */
+    static native void computeJobDestroy(long envPtr, long ptr);
+
+    /**
+     * Invoke local callback.
+     *
+     * @param envPtr Environment pointer.
+     * @param cbPtr Callback pointer.
+     * @param memPtr Memory pointer.
+     */
+    static native void continuousQueryListenerApply(long envPtr, long cbPtr, long memPtr);
+
+    /**
+     * Create filter in native platform.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     * @return Pointer to created filter.
+     */
+    static native long continuousQueryFilterCreate(long envPtr, long memPtr);
+
+    /**
+     * Invoke remote filter.
+     *
+     * @param envPtr Environment pointer.
+     * @param filterPtr Filter pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    static native int continuousQueryFilterApply(long envPtr, long filterPtr, long memPtr);
+
+    /**
+     * Release remote  filter.
+     *
+     * @param envPtr Environment pointer.
+     * @param filterPtr Filter pointer.
+     */
+    static native void continuousQueryFilterRelease(long envPtr, long filterPtr);
+
+    /**
+     * Notify native data streamer about topology update.
+     *
+     * @param envPtr Environment pointer.
+     * @param ptr Data streamer native pointer.
+     * @param topVer Topology version.
+     * @param topSize Topology size.
+     */
+    static native void dataStreamerTopologyUpdate(long envPtr, long ptr, long topVer, int topSize);
+
+    /**
+     * Invoke stream receiver.
+     *
+     * @param envPtr Environment pointer.
+     * @param ptr Receiver native pointer.
+     * @param cache Cache object.
+     * @param memPtr Stream pointer.
+     * @param keepPortable Portable flag.
+     */
+    static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, Object cache, long memPtr,
+        boolean keepPortable);
+
+    /**
+     * Notify future with byte result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureByteResult(long envPtr, long futPtr, int res);
+
+    /**
+     * Notify future with boolean result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureBoolResult(long envPtr, long futPtr, int res);
+
+    /**
+     * Notify future with short result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureShortResult(long envPtr, long futPtr, int res);
+
+    /**
+     * Notify future with byte result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureCharResult(long envPtr, long futPtr, int res);
+
+    /**
+     * Notify future with int result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureIntResult(long envPtr, long futPtr, int res);
+
+    /**
+     * Notify future with float result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureFloatResult(long envPtr, long futPtr, float res);
+
+    /**
+     * Notify future with long result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureLongResult(long envPtr, long futPtr, long res);
+
+    /**
+     * Notify future with double result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param res Result.
+     */
+    static native void futureDoubleResult(long envPtr, long futPtr, double res);
+
+    /**
+     * Notify future with object result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param memPtr Memory pointer.
+     */
+    static native void futureObjectResult(long envPtr, long futPtr, long memPtr);
+
+    /**
+     * Notify future with null result.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     */
+    static native void futureNullResult(long envPtr, long futPtr);
+
+    /**
+     * Notify future with error.
+     *
+     * @param envPtr Environment pointer.
+     * @param futPtr Future pointer.
+     * @param memPtr Pointer to memory with error information.
+     */
+    static native void futureError(long envPtr, long futPtr, long memPtr);
+
+    /**
+     * Creates message filter and returns a pointer.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    static native long messagingFilterCreate(long envPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    static native int messagingFilterApply(long envPtr, long objPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Pointer.
+     */
+    static native void messagingFilterDestroy(long envPtr, long objPtr);
+
+    /**
+     * Creates event filter and returns a pointer.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    static native long eventFilterCreate(long envPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Pointer.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     */
+    static native int eventFilterApply(long envPtr, long objPtr, long memPtr);
+
+    /**
+     * @param envPtr Environment pointer.
+     * @param objPtr Pointer.
+     */
+    static native void eventFilterDestroy(long envPtr, long objPtr);
+
+    /**
+     * Sends node info to native target.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Ptr to a stream with serialized node.
+     */
+    static native void nodeInfo(long envPtr, long memPtr);
+
+    /**
+     * Kernal start callback.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Memory pointer.
+     */
+    static native void onStart(long envPtr, long memPtr);
+
+    /*
+     * Kernal stop callback.
+     *
+     * @param envPtr Environment pointer.
+     */
+    static native void onStop(long envPtr);
+
+    /**
+     * Lifecycle event callback.
+     *
+     * @param envPtr Environment pointer.
+     * @param ptr Holder pointer.
+     * @param evt Event.
+     */
+    static native void lifecycleEvent(long envPtr, long ptr, int evt);
+
+    /**
+     * Re-allocate external memory chunk.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Cross-platform pointer.
+     * @param cap Capacity.
+     */
+    static native void memoryReallocate(long envPtr, long memPtr, int cap);
+
+    /**
+     * Initializes native service.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Stream pointer.
+     * @return Pointer to the native platform service.
+     */
+    static native long serviceInit(long envPtr, long memPtr);
+
+    /**
+     * Executes native service.
+     *
+     * @param envPtr Environment pointer.
+     * @param svcPtr Pointer to the service in the native platform.
+     * @param memPtr Stream pointer.
+     */
+    static native void serviceExecute(long envPtr, long svcPtr, long memPtr);
+
+    /**
+     * Cancels native service.
+     *
+     * @param envPtr Environment pointer.
+     * @param svcPtr Pointer to the service in the native platform.
+     * @param memPtr Stream pointer.
+     */
+    static native void serviceCancel(long envPtr, long svcPtr, long memPtr);
+
+    /**
+     /**
+     * Invokes service method.
+     *
+     * @param envPtr Environment pointer.
+     * @param svcPtr Pointer to the service in the native platform.
+     * @param outMemPtr Output memory pointer.
+     * @param inMemPtr Input memory pointer.
+     */
+    static native void serviceInvokeMethod(long envPtr, long svcPtr, long outMemPtr, long inMemPtr);
+
+    /**
+     * Invokes cluster node filter.
+     *
+     * @param envPtr Environment pointer.
+     * @param memPtr Stream pointer.
+     */
+    static native int clusterNodeFilterApply(long envPtr, long memPtr);
+
+    /**
+     * Private constructor.
+     */
+    private PlatformCallbackUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.java
new file mode 100644
index 0000000..9273e29
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.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.memory;
+
+import org.apache.ignite.internal.portable.streams.*;
+
+/**
+ * Interop output stream,
+ */
+public interface PlatformInputStream extends PortableInputStream {
+    /**
+     * Synchronize input. Must be called before start reading data from a memory changed by another platform.
+     */
+    public void synchronize();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.java
new file mode 100644
index 0000000..9d8f94e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.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.memory;
+
+/**
+ * Interop memory chunk.
+ */
+public interface PlatformMemory extends AutoCloseable {
+    /**
+     * Gets input stream.
+     *
+     * @return Input stream.
+     */
+    public PlatformInputStream input();
+
+    /**
+     * Gets output stream.
+     *
+     * @return Output stream.
+     */
+    public PlatformOutputStream output();
+
+    /**
+     * Gets pointer which can be passed between platforms.
+     *
+     * @return Pointer.
+     */
+    public long pointer();
+
+    /**
+     * Gets data pointer.
+     *
+     * @return Data pointer.
+     */
+    public long data();
+
+    /**
+     * Gets capacity.
+     *
+     * @return Capacity.
+     */
+    public int capacity();
+
+    /**
+     * Gets length.
+     *
+     * @return Length.
+     */
+    public int length();
+
+    /**
+     * Reallocate memory chunk.
+     *
+     * @param cap Minimum capacity.
+     */
+    public void reallocate(int cap);
+
+    /**
+     * Close memory releasing it.
+     */
+    @Override void close();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java
new file mode 100644
index 0000000..c2233a8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java
@@ -0,0 +1,46 @@
+/*
+ * 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.memory;
+
+/**
+ * Interop memory manager interface.
+ */
+public interface PlatformMemoryManager {
+    /**
+     * Allocates memory.
+     *
+     * @return Memory.
+     */
+    public PlatformMemory allocate();
+
+    /**
+     * Allocates memory having at least the given capacity.
+     *
+     * @param cap Minimum capacity.
+     * @return Memory.
+     */
+    public PlatformMemory allocate(int cap);
+
+    /**
+     * Gets memory from existing pointer.
+     *
+     * @param memPtr Cross-platform memory pointer.
+     * @return Memory.
+     */
+    public PlatformMemory get(long memPtr);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.java
new file mode 100644
index 0000000..eb2490a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.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.memory;
+
+import org.apache.ignite.internal.portable.streams.*;
+
+/**
+ * Interop output stream.
+ */
+public interface PlatformOutputStream extends PortableOutputStream {
+    /**
+     * Synchronize output stream with underlying memory
+     */
+    public void synchronize();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
deleted file mode 100644
index fb1eaa2..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ /dev/null
@@ -1,114 +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.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.portable.*;
-import org.apache.ignite.internal.processors.platform.callback.*;
-import org.apache.ignite.internal.processors.platform.memory.*;
-
-import java.util.*;
-
-/**
- * Platform context. Acts as an entry point for platform operations.
- */
-public interface PlatformContext {
-    /**
-     * Gets kernal context.
-     *
-     * @return Kernal context.
-     */
-    public GridKernalContext kernalContext();
-
-    /**
-     * Gets platform memory manager.
-     *
-     * @return Memory manager.
-     */
-    public PlatformMemoryManager memory();
-
-    /**
-     * Gets platform callback gateway.
-     *
-     * @return Callback gateway.
-     */
-    public PlatformCallbackGateway gateway();
-
-    /**
-     * Get memory reader.
-     *
-     * @param mem Memory.
-     * @return Reader.
-     */
-    public PortableRawReaderEx reader(PlatformMemory mem);
-
-    /**
-     * Get memory reader.
-     *
-     * @param in Input.
-     * @return Reader.
-     */
-    public PortableRawReaderEx reader(PlatformInputStream in);
-
-    /**
-     * Get memory writer.
-     *
-     * @param mem Memory.
-     * @return Writer.
-     */
-    public PortableRawWriterEx writer(PlatformMemory mem);
-
-    /**
-     * Get memory writer.
-     *
-     * @param out Output.
-     * @return Writer.
-     */
-    public PortableRawWriterEx writer(PlatformOutputStream out);
-
-    /**
-     * Sends node info to native platform, if necessary.
-     *
-     * @param node Node.
-     */
-    public void addNode(ClusterNode node);
-
-    /**
-     * Writes a node id to a stream and sends node info to native platform, if necessary.
-     *
-     * @param writer Writer.
-     * @param node Node.
-     */
-    public void writeNode(PortableRawWriterEx writer, ClusterNode node);
-
-    /**
-     * Writes multiple node ids to a stream and sends node info to native platform, if necessary.
-     *
-     * @param writer Writer.
-     * @param nodes Nodes.
-     */
-    public void writeNodes(PortableRawWriterEx writer, Collection<ClusterNode> nodes);
-
-    /**
-     * Process metadata from the platform.
-     *
-     * @param reader Reader.
-     */
-    public void processMetadata(PortableRawReaderEx reader);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
deleted file mode 100644
index a8e7879..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ /dev/null
@@ -1,869 +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.callback;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.*;
-
-/**
- * Gateway to all platform-dependent callbacks. Implementers might extend this class and provide additional callbacks.
- */
-@SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"})
-public class PlatformCallbackGateway {
-    /** Environment pointer. */
-    protected final long envPtr;
-
-    /** Lock. */
-    private final GridSpinReadWriteLock lock = new GridSpinReadWriteLock();
-
-    /**
-     * Native gateway.
-     *
-     * @param envPtr Environment pointer.
-     */
-    public PlatformCallbackGateway(long envPtr) {
-        this.envPtr = envPtr;
-    }
-
-    /**
-     * Get environment pointer.
-     *
-     * @return Environment pointer.
-     */
-    public long environmentPointer() {
-        return envPtr;
-    }
-
-    /**
-     * Create cache store.
-     *
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    public long cacheStoreCreate(long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.cacheStoreCreate(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param objPtr Object pointer.
-     * @param memPtr Memory pointer.
-     * @param cb Callback.
-     * @return Result.
-     */
-    public int cacheStoreInvoke(long objPtr, long memPtr, Object cb) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.cacheStoreInvoke(envPtr, objPtr, memPtr, cb);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param objPtr Object pointer.
-     */
-    public void cacheStoreDestroy(long objPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.cacheStoreDestroy(envPtr, objPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Creates cache store session.
-     *
-     * @param storePtr Store instance pointer.
-     * @return Session instance pointer.
-     */
-    public long cacheStoreSessionCreate(long storePtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.cacheStoreSessionCreate(envPtr, storePtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Creates cache entry filter and returns a pointer.
-     *
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    public long cacheEntryFilterCreate(long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.cacheEntryFilterCreate(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param ptr Pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    public int cacheEntryFilterApply(long ptr, long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.cacheEntryFilterApply(envPtr, ptr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param ptr Pointer.
-     */
-    public void cacheEntryFilterDestroy(long ptr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.cacheEntryFilterDestroy(envPtr, ptr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Invoke cache entry processor.
-     *
-     * @param outMemPtr Output memory pointer.
-     * @param inMemPtr Input memory pointer.
-     */
-    public void cacheInvoke(long outMemPtr, long inMemPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.cacheInvoke(envPtr, outMemPtr, inMemPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Perform native task map. Do not throw exceptions, serializing them to the output stream instead.
-     *
-     * @param taskPtr Task pointer.
-     * @param outMemPtr Output memory pointer (exists if topology changed, otherwise {@code 0}).
-     * @param inMemPtr Input memory pointer.
-     */
-    public void computeTaskMap(long taskPtr, long outMemPtr, long inMemPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.computeTaskMap(envPtr, taskPtr, outMemPtr, inMemPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Perform native task job result notification.
-     *
-     * @param taskPtr Task pointer.
-     * @param jobPtr Job pointer.
-     * @param memPtr Memory pointer (always zero for local job execution).
-     * @return Job result enum ordinal.
-     */
-    public int computeTaskJobResult(long taskPtr, long jobPtr, long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.computeTaskJobResult(envPtr, taskPtr, jobPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Perform native task reduce.
-     *
-     * @param taskPtr Task pointer.
-     */
-    public void computeTaskReduce(long taskPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.computeTaskReduce(envPtr, taskPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Complete task with native error.
-     *
-     * @param taskPtr Task pointer.
-     * @param memPtr Memory pointer with exception data or {@code 0} in case of success.
-     */
-    public void computeTaskComplete(long taskPtr, long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.computeTaskComplete(envPtr, taskPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Serialize native job.
-     *
-     * @param jobPtr Job pointer.
-     * @param memPtr Memory pointer.
-     * @return {@code True} if serialization succeeded.
-     */
-    public int computeJobSerialize(long jobPtr, long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.computeJobSerialize(envPtr, jobPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Create job in native platform.
-     *
-     * @param memPtr Memory pointer.
-     * @return Pointer to job.
-     */
-    public long computeJobCreate(long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.computeJobCreate(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Execute native job on a node other than where it was created.
-     *
-     * @param jobPtr Job pointer.
-     * @param cancel Cancel flag.
-     * @param memPtr Memory pointer to write result to for remote job execution or {@code 0} for local job execution.
-     */
-    public void computeJobExecute(long jobPtr, int cancel, long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.computeJobExecute(envPtr, jobPtr, cancel, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Cancel the job.
-     *
-     * @param jobPtr Job pointer.
-     */
-    public void computeJobCancel(long jobPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.computeJobCancel(envPtr, jobPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Destroy the job.
-     *
-     * @param ptr Pointer.
-     */
-    public void computeJobDestroy(long ptr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.computeJobDestroy(envPtr, ptr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Invoke local callback.
-     *
-     * @param cbPtr Callback pointer.
-     * @param memPtr Memory pointer.
-     */
-    public void continuousQueryListenerApply(long cbPtr, long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.continuousQueryListenerApply(envPtr, cbPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Create filter in native platform.
-     *
-     * @param memPtr Memory pointer.
-     * @return Pointer to created filter.
-     */
-    public long continuousQueryFilterCreate(long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.continuousQueryFilterCreate(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Invoke remote filter.
-     *
-     * @param filterPtr Filter pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    public int continuousQueryFilterApply(long filterPtr, long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.continuousQueryFilterApply(envPtr, filterPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Release remote  filter.
-     *
-     * @param filterPtr Filter pointer.
-     */
-    public void continuousQueryFilterRelease(long filterPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.continuousQueryFilterRelease(envPtr, filterPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify native data streamer about topology update.
-     *
-     * @param ptr Data streamer native pointer.
-     * @param topVer Topology version.
-     * @param topSize Topology size.
-     */
-    public void dataStreamerTopologyUpdate(long ptr, long topVer, int topSize) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.dataStreamerTopologyUpdate(envPtr, ptr, topVer, topSize);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Invoke stream receiver.
-     *
-     * @param ptr Receiver native pointer.
-     * @param cache Cache object.
-     * @param memPtr Stream pointer.
-     * @param keepPortable Portable flag.
-     */
-    public void dataStreamerStreamReceiverInvoke(long ptr, Object cache, long memPtr, boolean keepPortable) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.dataStreamerStreamReceiverInvoke(envPtr, ptr, cache, memPtr, keepPortable);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with byte result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureByteResult(long futPtr, int res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureByteResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with boolean result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureBoolResult(long futPtr, int res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureBoolResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with short result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureShortResult(long futPtr, int res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureShortResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with byte result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureCharResult(long futPtr, int res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureCharResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with int result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureIntResult(long futPtr, int res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureIntResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with float result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureFloatResult(long futPtr, float res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureFloatResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with long result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureLongResult(long futPtr, long res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureLongResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with double result.
-     *
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    public void futureDoubleResult(long futPtr, double res) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureDoubleResult(envPtr, futPtr, res);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with object result.
-     *
-     * @param futPtr Future pointer.
-     * @param memPtr Memory pointer.
-     */
-    public void futureObjectResult(long futPtr, long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureObjectResult(envPtr, futPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with null result.
-     *
-     * @param futPtr Future pointer.
-     */
-    public void futureNullResult(long futPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureNullResult(envPtr, futPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Notify future with error.
-     *
-     * @param futPtr Future pointer.
-     * @param memPtr Pointer to memory with error information.
-     */
-    public void futureError(long futPtr, long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.futureError(envPtr, futPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Creates message filter and returns a pointer.
-     *
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    public long messagingFilterCreate(long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.messagingFilterCreate(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param ptr Pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    public int messagingFilterApply(long ptr, long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.messagingFilterApply(envPtr, ptr, memPtr);
-        }
-        finally {
-            leave();
-        }}
-
-    /**
-     * @param ptr Pointer.
-     */
-    public void messagingFilterDestroy(long ptr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.messagingFilterDestroy(envPtr, ptr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Creates event filter and returns a pointer.
-     *
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    public long eventFilterCreate(long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.eventFilterCreate(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param ptr Pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    public int eventFilterApply(long ptr, long memPtr) {
-        enter();
-
-        try {
-            return PlatformCallbackUtils.eventFilterApply(envPtr, ptr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * @param ptr Pointer.
-     */
-    public void eventFilterDestroy(long ptr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.eventFilterDestroy(envPtr, ptr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Sends node info to native target.
-     *
-     * @param memPtr Ptr to a stream with serialized node.
-     */
-    public void nodeInfo(long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.nodeInfo(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Kernal start callback.
-     *
-     * @param memPtr Memory pointer.
-     */
-    public void onStart(long memPtr) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.onStart(envPtr, memPtr);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Lifecycle event callback.
-     *
-     * @param ptr Holder pointer.
-     * @param evt Event.
-     */
-    public void lifecycleEvent(long ptr, int evt) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.lifecycleEvent(envPtr, ptr, evt);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Re-allocate external memory chunk.
-     *
-     * @param memPtr Cross-platform pointer.
-     * @param cap Capacity.
-     */
-    public void memoryReallocate(long memPtr, int cap) {
-        enter();
-
-        try {
-            PlatformCallbackUtils.memoryReallocate(envPtr, memPtr, cap);
-        }
-        finally {
-            leave();
-        }
-    }
-
-    /**
-     * Initializes native service.
-     *
-     * @param memPtr Pointer.
-     * @throws org.apache.ignite.IgniteCheckedException In case of error.
-     */
-    public long serviceInit(long memPtr) throws IgniteCheckedException {
-        return PlatformCallbackUtils.serviceInit(envPtr, memPtr);
-    }
-
-    /**
-     * Executes native service.
-     *
-     * @param svcPtr Pointer to the service in the native platform.
-     * @param memPtr Stream pointer.
-     * @throws org.apache.ignite.IgniteCheckedException In case of error.
-     */
-    public void serviceExecute(long svcPtr, long memPtr) throws IgniteCheckedException {
-        PlatformCallbackUtils.serviceExecute(envPtr, svcPtr, memPtr);
-    }
-
-    /**
-     * Cancels native service.
-     *
-     * @param svcPtr Pointer to the service in the native platform.
-     * @param memPtr Stream pointer.
-     * @throws org.apache.ignite.IgniteCheckedException In case of error.
-     */
-    public void serviceCancel(long svcPtr, long memPtr) throws IgniteCheckedException {
-        PlatformCallbackUtils.serviceCancel(envPtr, svcPtr, memPtr);
-    }
-
-    /**
-     * Invokes service method.
-     *
-     * @param svcPtr Pointer to the service in the native platform.
-     * @param outMemPtr Output memory pointer.
-     * @param inMemPtr Input memory pointer.
-     * @throws org.apache.ignite.IgniteCheckedException In case of error.
-     */
-    public void serviceInvokeMethod(long svcPtr, long outMemPtr, long inMemPtr) throws IgniteCheckedException {
-        PlatformCallbackUtils.serviceInvokeMethod(envPtr, svcPtr, outMemPtr, inMemPtr);
-    }
-
-    /**
-     * Invokes cluster node filter.
-     *
-     * @param memPtr Stream pointer.
-     */
-    public int clusterNodeFilterApply(long memPtr) {
-        return PlatformCallbackUtils.clusterNodeFilterApply(envPtr, memPtr);
-    }
-
-    /**
-     * Kernal stop callback.
-     */
-    public void onStop() {
-        block();
-
-        PlatformCallbackUtils.onStop(envPtr);
-    }
-
-    /**
-     * Enter gateway.
-     */
-    protected void enter() {
-        if (!lock.tryReadLock())
-            throw new IgniteException("Failed to execute native callback because grid is stopping.");
-    }
-
-    /**
-     * Leave gateway.
-     */
-    protected void leave() {
-        lock.readUnlock();
-    }
-
-    /**
-     * Block gateway.
-     */
-    protected void block() {
-        lock.writeLock();
-    }
-}


[43/50] [abbrv] ignite git commit: Moved platform CachePartialUpdateException to Ignite.

Posted by ak...@apache.org.
Moved platform CachePartialUpdateException to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 26f0ee0ad7ce9df573dd180a96ff0bf2a366495f
Parents: 63ac8cd
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 13:51:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 13:51:08 2015 +0300

----------------------------------------------------------------------
 .../PlatformCachePartialUpdateException.java    | 68 ++++++++++++++++++++
 1 file changed, 68 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/26f0ee0a/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
new file mode 100644
index 0000000..925b0b2
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
@@ -0,0 +1,68 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+
+import java.util.*;
+
+/**
+ * Interop cache partial update exception.
+ */
+public class PlatformCachePartialUpdateException extends PlatformException implements PlatformExtendedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Platform context. */
+    private final PlatformContext ctx;
+
+    /** Keep portable flag. */
+    private final boolean keepPortable;
+
+    /**
+     * Constructor.
+     *
+     * @param cause Root cause.
+     * @param ctx Context.
+     * @param keepPortable Keep portable flag.
+     */
+    public PlatformCachePartialUpdateException(CachePartialUpdateException cause, PlatformContext ctx,
+        boolean keepPortable) {
+        super(cause);
+
+        this.ctx = ctx;
+        this.keepPortable = keepPortable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformContext context() {
+        return ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeData(PortableRawWriterEx writer) {
+        Collection keys = ((CachePartialUpdateException)getCause()).failedKeys();
+
+        writer.writeBoolean(keepPortable);
+
+        PlatformUtils.writeNullableCollection(writer, keys);
+    }
+}


[37/50] [abbrv] ignite git commit: IGNITE-1312: Moved continuous queries to Ignite.

Posted by ak...@apache.org.
IGNITE-1312: Moved continuous queries to Ignite.


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

Branch: refs/heads/ignite-843
Commit: f4c7107ce90f54f6cff7cd83d18b025011b2c7cf
Parents: 3e30c86
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 09:24:25 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 09:24:25 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |  19 ++
 .../processors/platform/PlatformTarget.java     |  76 +++++++
 .../cache/query/PlatformContinuousQuery.java    |  58 +++++
 .../processors/platform/PlatformTarget.java     |  76 -------
 .../query/PlatformContinuousQueryImpl.java      | 222 +++++++++++++++++++
 .../PlatformContinuousQueryRemoteFilter.java    | 183 +++++++++++++++
 6 files changed, 558 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f4c7107c/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 504f79e..461fb84 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
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.platform;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.query.continuous.*;
+import org.apache.ignite.internal.processors.platform.cache.query.*;
 import org.apache.ignite.internal.processors.platform.callback.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
 import org.jetbrains.annotations.*;
@@ -135,4 +137,21 @@ public interface PlatformContext {
      * @param metrics Metrics.
      */
     public void writeClusterMetrics(PortableRawWriterEx writer, @Nullable ClusterMetrics metrics);
+
+    /**
+     *
+     * @param ptr Pointer to continuous query deployed on the platform.
+     * @param hasFilter Whether filter exists.
+     * @param filter Filter.
+     * @return Platform continuous query.
+     */
+    public PlatformContinuousQuery createContinuousQuery(long ptr, boolean hasFilter, @Nullable Object filter);
+
+    /**
+     * Create continuous query filter to be deployed on remote node.
+     *
+     * @param filter Native filter.
+     * @return Filter.
+     */
+    public CacheContinuousQueryFilterEx createContinuousQueryFilter(Object filter);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4c7107c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
new file mode 100644
index 0000000..1d54b4e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jetbrains.annotations.*;
+
+/**
+ * Interop target abstraction.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public interface PlatformTarget {
+    /**
+     * Synchronous IN operation.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @return Value specific for the given operation otherwise.
+     * @throws Exception If failed.
+     */
+    public int inOp(int type, long memPtr) throws Exception;
+
+    /**
+     * Synchronous IN operation which returns managed object as result.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @return Managed result.
+     * @throws Exception If case of failure.
+     */
+    public Object inOpObject(int type, long memPtr) throws Exception;
+
+    /**
+     * Synchronous OUT operation.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @throws Exception In case of failure.
+     */
+    public void outOp(int type, long memPtr) throws Exception;
+
+    /**
+     * Synchronous IN-OUT operation.
+     *
+     * @param type Operation type.
+     * @param inMemPtr Input memory pointer.
+     * @param outMemPtr Output memory pointer.
+     * @throws Exception In case of failure.
+     */
+    public void inOutOp(int type, long inMemPtr, long outMemPtr) throws Exception;
+
+    /**
+     * Synchronous IN-OUT operation with optional argument.
+     *
+     * @param type Operation type.
+     * @param inMemPtr Input memory pointer.
+     * @param outMemPtr Output memory pointer.
+     * @param arg Argument (optional).
+     * @throws Exception In case of failure.
+     */
+    public void inOutOp(int type, long inMemPtr, long outMemPtr, @Nullable Object arg) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4c7107c/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
new file mode 100644
index 0000000..0b55aea
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQuery.java
@@ -0,0 +1,58 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.continuous.*;
+import org.apache.ignite.internal.processors.platform.*;
+
+import javax.cache.event.*;
+
+/**
+ * Platform continuous query.
+ */
+public interface PlatformContinuousQuery extends CacheEntryUpdatedListener, CacheContinuousQueryFilterEx {
+    /**
+     * Start continuous query execution.
+     *
+     * @param cache Cache.
+     * @param loc Local flag.
+     * @param bufSize Buffer size.
+     * @param timeInterval Time interval.
+     * @param autoUnsubscribe Auto-unsubscribe flag.
+     * @param initialQry Initial query.
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    public void start(IgniteCacheProxy cache, boolean loc, int bufSize, long timeInterval, boolean autoUnsubscribe,
+        Query initialQry) throws IgniteCheckedException;
+
+    /**
+     * Close continuous query.
+     */
+    public void close();
+
+    /**
+     * Gets initial query cursor (if any).
+     *
+     * @return Initial query cursor.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public PlatformTarget getInitialQueryCursor();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4c7107c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
deleted file mode 100644
index 1d54b4e..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
+++ /dev/null
@@ -1,76 +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.jetbrains.annotations.*;
-
-/**
- * Interop target abstraction.
- */
-@SuppressWarnings("UnusedDeclaration")
-public interface PlatformTarget {
-    /**
-     * Synchronous IN operation.
-     *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
-     * @return Value specific for the given operation otherwise.
-     * @throws Exception If failed.
-     */
-    public int inOp(int type, long memPtr) throws Exception;
-
-    /**
-     * Synchronous IN operation which returns managed object as result.
-     *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
-     * @return Managed result.
-     * @throws Exception If case of failure.
-     */
-    public Object inOpObject(int type, long memPtr) throws Exception;
-
-    /**
-     * Synchronous OUT operation.
-     *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
-     * @throws Exception In case of failure.
-     */
-    public void outOp(int type, long memPtr) throws Exception;
-
-    /**
-     * Synchronous IN-OUT operation.
-     *
-     * @param type Operation type.
-     * @param inMemPtr Input memory pointer.
-     * @param outMemPtr Output memory pointer.
-     * @throws Exception In case of failure.
-     */
-    public void inOutOp(int type, long inMemPtr, long outMemPtr) throws Exception;
-
-    /**
-     * Synchronous IN-OUT operation with optional argument.
-     *
-     * @param type Operation type.
-     * @param inMemPtr Input memory pointer.
-     * @param outMemPtr Output memory pointer.
-     * @param arg Argument (optional).
-     * @throws Exception In case of failure.
-     */
-    public void inOutOp(int type, long inMemPtr, long outMemPtr, @Nullable Object arg) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4c7107c/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
new file mode 100644
index 0000000..b2fa1e3
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
@@ -0,0 +1,222 @@
+/*
+ *  Copyright (C) GridGain Systems. All Rights Reserved.
+ *  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.internal.processors.platform.cache.query;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.processors.query.*;
+
+import javax.cache.*;
+import javax.cache.event.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Interop continuous query handle.
+ */
+public class PlatformContinuousQueryImpl implements PlatformContinuousQuery {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Context. */
+    protected final PlatformContext platformCtx;
+
+    /** Whether filter exists. */
+    private final boolean hasFilter;
+
+    /** Native filter in serialized form. If null, then filter is either not set, or this is local query. */
+    protected final Object filter;
+
+    /** Pointer to native counterpart; zero if closed. */
+    private long ptr;
+
+    /** Cursor to handle filter close. */
+    private QueryCursor cursor;
+
+    /** Lock for concurrency control. */
+    private final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /** Wrapped initial qry cursor. */
+    private PlatformQueryCursor initialQryCur;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param ptr Pointer to native counterpart.
+     * @param hasFilter Whether filter exists.
+     * @param filter Filter.
+     */
+    public PlatformContinuousQueryImpl(PlatformContext platformCtx, long ptr, boolean hasFilter, Object filter) {
+        assert ptr != 0L;
+
+        this.platformCtx = platformCtx;
+        this.ptr = ptr;
+        this.hasFilter = hasFilter;
+        this.filter = filter;
+    }
+
+    /**
+     * Start query execution.
+     *
+     * @param cache Cache.
+     * @param loc Local flag.
+     * @param bufSize Buffer size.
+     * @param timeInterval Time interval.
+     * @param autoUnsubscribe Auto-unsubscribe flag.
+     * @param initialQry Initial query.
+     */
+    @SuppressWarnings("unchecked")
+    public void start(IgniteCacheProxy cache, boolean loc, int bufSize, long timeInterval, boolean autoUnsubscribe,
+        Query initialQry) throws IgniteCheckedException {
+        assert !loc || filter == null;
+
+        lock.writeLock().lock();
+
+        try {
+            try {
+                ContinuousQuery qry = new ContinuousQuery();
+
+                qry.setLocalListener(this);
+                qry.setRemoteFilter(this); // Filter must be set always for correct resource release.
+                qry.setPageSize(bufSize);
+                qry.setTimeInterval(timeInterval);
+                qry.setAutoUnsubscribe(autoUnsubscribe);
+                qry.setInitialQuery(initialQry);
+
+                cursor = cache.query(qry.setLocal(loc));
+
+                if (initialQry != null)
+                    initialQryCur = new PlatformQueryCursor(platformCtx, new QueryCursorEx<Cache.Entry>() {
+                        @Override public Iterator<Cache.Entry> iterator() {
+                            return cursor.iterator();
+                        }
+
+                        @Override public List<Cache.Entry> getAll() {
+                            return cursor.getAll();
+                        }
+
+                        @Override public void close() {
+                            // No-op: do not close whole continuous query when initial query cursor closes.
+                        }
+
+                        @Override public void getAll(Consumer<Cache.Entry> clo) throws IgniteCheckedException {
+                            for (Cache.Entry t : this)
+                                clo.consume(t);
+                        }
+
+                        @Override public List<GridQueryFieldMetadata> fieldsMeta() {
+                            return null;
+                        }
+                    }, initialQry.getPageSize() > 0 ? initialQry.getPageSize() : Query.DFLT_PAGE_SIZE);
+            }
+            catch (Exception e) {
+                try
+                {
+                    close0();
+                }
+                catch (Exception ignored)
+                {
+                    // Ignore
+                }
+
+                throw PlatformUtils.unwrapQueryException(e);
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void onUpdated(Iterable evts) throws CacheEntryListenerException {
+        lock.readLock().lock();
+
+        try {
+            if (ptr == 0)
+                throw new CacheEntryListenerException("Failed to notify listener because it has been closed.");
+
+            PlatformUtils.applyContinuousQueryEvents(platformCtx, ptr, evts);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean evaluate(CacheEntryEvent evt) throws CacheEntryListenerException {
+        lock.readLock().lock();
+
+        try {
+            if (ptr == 0)
+                throw new CacheEntryListenerException("Failed to evaluate the filter because it has been closed.");
+
+            return !hasFilter || PlatformUtils.evaluateContinuousQueryEvent(platformCtx, ptr, evt);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onQueryUnregister() {
+        close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        lock.writeLock().lock();
+
+        try {
+            close0();
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"UnusedDeclaration", "unchecked"})
+    @Override public PlatformTarget getInitialQueryCursor() {
+        return initialQryCur;
+    }
+
+    /**
+     * Internal close routine.
+     */
+    private void close0() {
+        if (ptr != 0) {
+            long ptr0 = ptr;
+
+            ptr = 0;
+
+            if (cursor != null)
+                cursor.close();
+
+            platformCtx.gateway().continuousQueryFilterRelease(ptr0);
+        }
+    }
+
+    /**
+     * Replacer for remote filter.
+     *
+     * @return Filter to be deployed on remote node.
+     * @throws ObjectStreamException If failed.
+     */
+    Object writeReplace() throws ObjectStreamException {
+        return filter == null ? null : platformCtx.createContinuousQueryFilter(filter);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f4c7107c/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
new file mode 100644
index 0000000..0f19218
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
@@ -0,0 +1,183 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.cache.query.continuous.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+
+import javax.cache.event.*;
+import java.io.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Continuous query filter deployed on remote nodes.
+ */
+public class PlatformContinuousQueryRemoteFilter implements CacheContinuousQueryFilterEx, Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Lock for concurrency control. */
+    private final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /** Native filter in serialized form. */
+    private Object filter;
+
+    /** Grid hosting the filter. */
+    @IgniteInstanceResource
+    private transient Ignite grid;
+
+    /** Native platform pointer. */
+    private transient volatile long ptr;
+
+    /** Close flag. Once set, none requests to native platform is possible. */
+    private transient boolean closed;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    public PlatformContinuousQueryRemoteFilter() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param filter Serialized native filter.
+     */
+    public PlatformContinuousQueryRemoteFilter(Object filter) {
+        assert filter != null;
+
+        this.filter = filter;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean evaluate(CacheEntryEvent evt) throws CacheEntryListenerException {
+        long ptr0 = ptr;
+
+        if (ptr0 == 0)
+            deploy();
+
+        lock.readLock().lock();
+
+        try {
+            if (closed)
+                throw new CacheEntryListenerException("Failed to evaluate the filter because it has been closed.");
+
+            PlatformContext platformCtx = PlatformUtils.platformContext(grid);
+
+            return PlatformUtils.evaluateContinuousQueryEvent(platformCtx, ptr, evt);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Deploy filter to native platform.
+     */
+    private void deploy() {
+        lock.writeLock().lock();
+
+        try {
+            // 1. Do not deploy if the filter has been closed concurrently.
+            if (closed)
+                throw new CacheEntryListenerException("Failed to deploy the filter because it has been closed.");
+
+            // 2. Deploy.
+            PlatformContext ctx = PlatformUtils.platformContext(grid);
+
+            try (PlatformMemory mem = ctx.memory().allocate()) {
+                PlatformOutputStream out = mem.output();
+
+                PortableRawWriterEx writer = ctx.writer(out);
+
+                writer.writeObject(filter);
+
+                out.synchronize();
+
+                ptr = ctx.gateway().continuousQueryFilterCreate(mem.pointer());
+            }
+            catch (Exception e) {
+                // 3. Close in case of failure.
+                close();
+
+                throw new CacheEntryListenerException("Failed to deploy the filter.", e);
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onQueryUnregister() {
+        lock.writeLock().lock();
+
+        try {
+            close();
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Close the filter.
+     */
+    private void close() {
+        if (!closed) {
+            try {
+                if (ptr != 0) {
+                    try {
+                        PlatformUtils.platformContext(grid).gateway().continuousQueryFilterRelease(ptr);
+                    }
+                    finally {
+                        // Nullify the pointer in any case.
+                        ptr = 0;
+                    }
+                }
+            }
+            finally {
+                closed = true;
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(filter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        filter = in.readObject();
+
+        assert filter != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformContinuousQueryRemoteFilter.class, this);
+    }
+}


[24/50] [abbrv] 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/00b27cec
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/00b27cec
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/00b27cec

Branch: refs/heads/ignite-843
Commit: 00b27cecb917edf61573cf2989e268636d9fb312
Parents: e567f8c 58f9fe4
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 12:44:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 12:44:38 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteInternalFuture.java   | 10 +++
 .../query/GridCacheQueryFutureAdapter.java      |  7 ++
 .../processors/igfs/IgfsMetaManager.java        | 43 +++++------
 .../internal/processors/igfs/IgfsUtils.java     | 52 +++++++++++++
 .../util/future/GridFinishedFuture.java         |  5 ++
 .../internal/util/future/GridFutureAdapter.java | 58 +++++++++------
 .../platform/PlatformAbstractPredicate.java     | 64 ++++++++++++++++
 .../processors/platform/PlatformBootstrap.java  |  1 -
 .../cluster/PlatformClusterNodeFilter.java      | 77 ++++++++++++++++++++
 scripts/apply-pull-request.sh                   |  2 +
 10 files changed, 274 insertions(+), 45 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] ignite git commit: correct predefined classes IDs mapping (portable context)

Posted by ak...@apache.org.
correct predefined classes IDs mapping (portable context)


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

Branch: refs/heads/ignite-843
Commit: 28c8dc7e9d92fb95aa1a9bdef82944697aac95e6
Parents: 3d46b62
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Aug 27 13:57:58 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Aug 27 13:57:58 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/portable/PortableContext.java  | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/28c8dc7e/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 723113e..cd3abc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -159,6 +159,8 @@ public class PortableContext implements Externalizable {
         mapTypes.put(ConcurrentHashMap8.class, GridPortableMarshaller.CONC_HASH_MAP);
         mapTypes.put(Properties.class, GridPortableMarshaller.PROPERTIES_MAP);
 
+        // IDs range from [0..200] is used by Java SDK API and GridGain legacy API
+
         registerPredefinedType(Byte.class, GridPortableMarshaller.BYTE);
         registerPredefinedType(Boolean.class, GridPortableMarshaller.BOOLEAN);
         registerPredefinedType(Short.class, GridPortableMarshaller.SHORT);
@@ -205,9 +207,10 @@ public class PortableContext implements Externalizable {
         registerPredefinedType(IgniteBiTuple.class, 61);
         registerPredefinedType(T2.class, 62);
 
-        registerPredefinedType(PortableObjectImpl.class, 63);
+        // IDs range [200..1000] is used by Ignite internal APIs.
 
-        registerPredefinedType(PortableMetaDataImpl.class, 64);
+        registerPredefinedType(PortableObjectImpl.class, 200);
+        registerPredefinedType(PortableMetaDataImpl.class, 201);
     }
 
     /**


[42/50] [abbrv] ignite git commit: Fixing license.

Posted by ak...@apache.org.
Fixing license.


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

Branch: refs/heads/ignite-843
Commit: 63ac8cd28cbb5bb0c8f6216864222051bb889782
Parents: 5bbb8a3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 13:45:44 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 13:45:44 2015 +0300

----------------------------------------------------------------------
 .../query/PlatformContinuousQueryImpl.java      | 20 ++++++++++++++------
 1 file changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/63ac8cd2/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
index b2fa1e3..d3af446 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryImpl.java
@@ -1,10 +1,18 @@
 /*
- *  Copyright (C) GridGain Systems. All Rights Reserved.
- *  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ * 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;


[18/50] [abbrv] 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/6b2ee50b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6b2ee50b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6b2ee50b

Branch: refs/heads/ignite-843
Commit: 6b2ee50b058e37f71ec1a0aab76454aaef87dec2
Parents: 9fe3e8f c8bc1f9
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 12:11:45 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 12:11:45 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 23 ++++++++++++++++++--
 .../testframework/junits/GridAbstractTest.java  |  2 +-
 2 files changed, 22 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[22/50] [abbrv] 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/58f9fe42
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/58f9fe42
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/58f9fe42

Branch: refs/heads/ignite-843
Commit: 58f9fe425d4eff652d459bec722dba3c22e5e03a
Parents: 93b2942 a2b7ba1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 12:26:23 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 12:26:23 2015 +0300

----------------------------------------------------------------------
 scripts/apply-pull-request.sh | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------



[08/50] [abbrv] ignite git commit: Restored backward compatibility for cache objects

Posted by ak...@apache.org.
Restored backward compatibility for cache objects


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

Branch: refs/heads/ignite-843
Commit: e428206d3d2281de72e674fe5abf9f887bf221fc
Parents: 38b3ffd
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Aug 26 15:50:34 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Aug 26 15:50:34 2015 -0700

----------------------------------------------------------------------
 .../cache/portable/CacheObjectPortableProcessorImpl.java          | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e428206d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
index a421129..e70feb9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.portable.*;
+import org.apache.ignite.plugin.*;
 import org.apache.ignite.portable.*;
 
 import org.jetbrains.annotations.*;
@@ -569,7 +570,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isPortableEnabled() {
+    @Override public boolean isPortableEnabled(CacheConfiguration<?, ?> ccfg) {
         return marsh instanceof PortableMarshaller;
     }
 


[14/50] [abbrv] ignite git commit: Use special method for test debug info dumping to avoid waiting for hanging nodes start.

Posted by ak...@apache.org.
Use special method  for test debug info dumping to avoid waiting for hanging nodes start.


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

Branch: refs/heads/ignite-843
Commit: fc60fee5f727210f5a3a16997e2bd6b7b07a4539
Parents: 5877b30
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 12:07:47 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 12:07:47 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 23 ++++++++++++++++++--
 .../testframework/junits/GridAbstractTest.java  |  2 +-
 2 files changed, 22 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fc60fee5/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index fd74745..cd91fa8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1015,10 +1015,29 @@ public class IgnitionEx {
      * @return List of all grids started so far.
      */
     public static List<Ignite> allGrids() {
+        return allGrids(true);
+    }
+
+    /**
+     * Gets a list of all grids started so far.
+     *
+     * @return List of all grids started so far.
+     */
+    public static List<Ignite> allGridsx() {
+        return allGrids(false);
+    }
+
+    /**
+     * Gets a list of all grids started so far.
+     *
+     * @param wait If {@code true} wait for node start finish.
+     * @return List of all grids started so far.
+     */
+    private static List<Ignite> allGrids(boolean wait) {
         List<Ignite> allIgnites = new ArrayList<>(grids.size() + 1);
 
         for (IgniteNamedInstance grid : grids.values()) {
-            Ignite g = grid.grid();
+            Ignite g = wait ? grid.grid() : grid.gridx();
 
             if (g != null)
                 allIgnites.add(g);
@@ -1027,7 +1046,7 @@ public class IgnitionEx {
         IgniteNamedInstance dfltGrid0 = dfltGrid;
 
         if (dfltGrid0 != null) {
-            IgniteKernal g = dfltGrid0.grid();
+            IgniteKernal g = wait ? dfltGrid0.grid() : dfltGrid0.gridx();
 
             if (g != null)
                 allIgnites.add(g);

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc60fee5/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index d6591cd..9cd621a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -1578,7 +1578,7 @@ public abstract class GridAbstractTest extends TestCase {
                 "Test has been timed out and will be interrupted (threads dump will be taken before interruption) [" +
                 "test=" + getName() + ", timeout=" + getTestTimeout() + ']');
 
-            List<Ignite> nodes = G.allGrids();
+            List<Ignite> nodes = IgnitionEx.allGridsx();
 
             for (Ignite node : nodes)
                 ((IgniteKernal)node).dumpDebugInfo();


[28/50] [abbrv] 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/f497e8e2
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/f497e8e2
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/f497e8e2

Branch: refs/heads/ignite-843
Commit: f497e8e2d36b93c9ee4e4b810478b76c0b4ff585
Parents: f575ff1 7c2c02b
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Aug 27 13:21:46 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Aug 27 13:21:46 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java         | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------



[46/50] [abbrv] ignite git commit: IGNITE-1298: Now output stream flush awaits for all data blocks to be processed before updating file length in meta cache.

Posted by ak...@apache.org.
IGNITE-1298: Now output stream flush awaits for all data blocks to be processed before updating file length in meta cache.


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

Branch: refs/heads/ignite-843
Commit: 02f246535dd31346ff94485810e3bb306bb67cbb
Parents: 16c095a
Author: iveselovskiy <iv...@gridgain.com>
Authored: Fri Aug 28 16:03:45 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 16:03:45 2015 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDataManager.java        | 169 +++++++++----------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 +
 .../igfs/IgfsBackupFailoverSelfTest.java        | 137 +++++++++++++--
 3 files changed, 203 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/02f24653/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index aa6427d..602924d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -32,7 +32,6 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.datastreamer.*;
 import org.apache.ignite.internal.processors.task.*;
-import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -323,58 +322,6 @@ public class IgfsDataManager extends IgfsManager {
     }
 
     /**
-     * Get list of local data blocks of the given file.
-     *
-     * @param fileInfo File info.
-     * @return List of local data block indices.
-     * @throws IgniteCheckedException If failed.
-     */
-    public List<Long> listLocalDataBlocks(IgfsFileInfo fileInfo)
-        throws IgniteCheckedException {
-        assert fileInfo != null;
-
-        int prevGrpIdx = 0; // Block index within affinity group.
-
-        boolean prevPrimaryFlag = false; // Whether previous block was primary.
-
-        List<Long> res = new ArrayList<>();
-
-        for (long i = 0; i < fileInfo.blocksCount(); i++) {
-            // Determine group index.
-            int grpIdx = (int)(i % grpSize);
-
-            if (prevGrpIdx < grpIdx) {
-                // Reuse existing affinity result.
-                if (prevPrimaryFlag)
-                    res.add(i);
-            }
-            else {
-                // Re-calculate affinity result.
-                IgfsBlockKey key = new IgfsBlockKey(fileInfo.id(), fileInfo.affinityKey(),
-                    fileInfo.evictExclude(), i);
-
-                Collection<ClusterNode> affNodes = dataCache.affinity().mapKeyToPrimaryAndBackups(key);
-
-                assert affNodes != null && !affNodes.isEmpty();
-
-                ClusterNode primaryNode = affNodes.iterator().next();
-
-                if (primaryNode.id().equals(igfsCtx.kernalContext().localNodeId())) {
-                    res.add(i);
-
-                    prevPrimaryFlag = true;
-                }
-                else
-                    prevPrimaryFlag = false;
-            }
-
-            prevGrpIdx = grpIdx;
-        }
-
-        return res;
-    }
-
-    /**
      * Get data block for specified file ID and block index.
      *
      * @param fileInfo File info.
@@ -1764,6 +1711,19 @@ public class IgfsDataManager extends IgfsManager {
     }
 
     /**
+     * Allows output stream to await for all current acks.
+     *
+     * @param fileId File ID.
+     * @throws IgniteInterruptedCheckedException In case of interrupt.
+     */
+    void awaitAllAcksReceived(IgniteUuid fileId) throws IgniteInterruptedCheckedException {
+        WriteCompletionFuture fut = pendingWrites.get(fileId);
+
+        if (fut != null)
+            fut.awaitAllAcksReceived();
+    }
+
+    /**
      * Future that is completed when all participating
      */
     private class WriteCompletionFuture extends GridFutureAdapter<Boolean> {
@@ -1771,10 +1731,16 @@ public class IgfsDataManager extends IgfsManager {
         private static final long serialVersionUID = 0L;
 
         /** File id to remove future from map. */
-        private IgniteUuid fileId;
+        private final IgniteUuid fileId;
 
         /** Pending acks. */
-        private ConcurrentMap<UUID, Set<Long>> pendingAcks = new ConcurrentHashMap8<>();
+        private final ConcurrentMap<Long, UUID> ackMap = new ConcurrentHashMap8<>();
+
+        /** Lock for map-related conditions. */
+        private final Lock lock = new ReentrantLock();
+
+        /** Condition to wait for empty map. */
+        private final Condition allAcksRcvCond = lock.newCondition();
 
         /** Flag indicating future is waiting for last ack. */
         private volatile boolean awaitingLast;
@@ -1788,6 +1754,23 @@ public class IgfsDataManager extends IgfsManager {
             this.fileId = fileId;
         }
 
+        /**
+         * Await all pending data blockes to be acked.
+         *
+         * @throws IgniteInterruptedCheckedException In case of interrupt.
+         */
+        public void awaitAllAcksReceived() throws IgniteInterruptedCheckedException {
+            lock.lock();
+
+            try {
+                while (!ackMap.isEmpty())
+                    U.await(allAcksRcvCond);
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+
         /** {@inheritDoc} */
         @Override public boolean onDone(@Nullable Boolean res, @Nullable Throwable err) {
             if (!isDone()) {
@@ -1808,26 +1791,41 @@ public class IgfsDataManager extends IgfsManager {
          */
         private void onWriteRequest(UUID nodeId, long batchId) {
             if (!isDone()) {
-                Set<Long> reqIds = pendingAcks.get(nodeId);
+                UUID pushedOut = ackMap.putIfAbsent(batchId, nodeId);
 
-                if (reqIds == null)
-                    reqIds = F.addIfAbsent(pendingAcks, nodeId, new GridConcurrentHashSet<Long>());
-
-                reqIds.add(batchId);
+                assert pushedOut == null;
             }
         }
 
         /**
+         * Answers if there are some batches for the specified node we're currently waiting acks for.
+         *
+         * @param nodeId The node Id.
+         * @return If there are acks awaited from this node.
+         */
+        private boolean hasPendingAcks(UUID nodeId) {
+            assert nodeId != null;
+
+            for (Map.Entry<Long, UUID> e : ackMap.entrySet())
+                if (nodeId.equals(e.getValue()))
+                    return true;
+
+            return false;
+        }
+
+        /**
          * Error occurred on node with given ID.
          *
          * @param nodeId Node ID.
          * @param e Caught exception.
          */
         private void onError(UUID nodeId, IgniteCheckedException e) {
-            Set<Long> reqIds = pendingAcks.get(nodeId);
-
             // If waiting for ack from this node.
-            if (reqIds != null && !reqIds.isEmpty()) {
+            if (hasPendingAcks(nodeId)) {
+                ackMap.clear();
+
+                signalNoAcks();
+
                 if (e.hasCause(IgfsOutOfSpaceException.class))
                     onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " + nodeId, e));
                 else
@@ -1844,18 +1842,31 @@ public class IgfsDataManager extends IgfsManager {
          */
         private void onWriteAck(UUID nodeId, long batchId) {
             if (!isDone()) {
-                Set<Long> reqIds = pendingAcks.get(nodeId);
+                boolean rmv = ackMap.remove(batchId, nodeId);
 
-                assert reqIds != null : "Received acknowledgement message for not registered node [nodeId=" +
+                assert rmv : "Received acknowledgement message for not registered batch [nodeId=" +
                     nodeId + ", batchId=" + batchId + ']';
 
-                boolean rmv = reqIds.remove(batchId);
+                if (ackMap.isEmpty()) {
+                    signalNoAcks();
 
-                assert rmv : "Received acknowledgement message for not registered batch [nodeId=" +
-                    nodeId + ", batchId=" + batchId + ']';
+                    if (awaitingLast)
+                        onDone(true);
+                }
+            }
+        }
+
+        /**
+         * Signal that currenlty there are no more pending acks.
+         */
+        private void signalNoAcks() {
+            lock.lock();
 
-                if (awaitingLast && checkCompleted())
-                    onDone(true);
+            try {
+                allAcksRcvCond.signalAll();
+            }
+            finally {
+                lock.unlock();
             }
         }
 
@@ -1868,24 +1879,8 @@ public class IgfsDataManager extends IgfsManager {
             if (log.isDebugEnabled())
                 log.debug("Marked write completion future as awaiting last ack: " + fileId);
 
-            if (checkCompleted())
+            if (ackMap.isEmpty())
                 onDone(true);
         }
-
-        /**
-         * @return True if received all request acknowledgements after {@link #markWaitingLastAck()} was called.
-         */
-        private boolean checkCompleted() {
-            for (Map.Entry<UUID, Set<Long>> entry : pendingAcks.entrySet()) {
-                Set<Long> reqIds = entry.getValue();
-
-                // If still waiting for some acks.
-                if (!reqIds.isEmpty())
-                    return false;
-            }
-
-            // Got match for each entry in sent map.
-            return true;
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/02f24653/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index 298733a..01359b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -283,6 +283,8 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
             }
 
             if (space > 0) {
+                data.awaitAllAcksReceived(fileInfo.id());
+
                 IgfsFileInfo fileInfo0 = meta.updateInfo(fileInfo.id(),
                     new ReserveSpaceClosure(space, streamRange));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/02f24653/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
index 0162121..09cecaa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.secondary.*;
+import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
@@ -108,21 +109,6 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * Creates IPC configuration.
-     *
-     * @param port The port to use.
-     * @return The endpoint configuration.
-     */
-    protected IgfsIpcEndpointConfiguration createIgfsRestConfig(int port) {
-        IgfsIpcEndpointConfiguration cfg = new IgfsIpcEndpointConfiguration();
-
-        cfg.setType(IgfsIpcEndpointType.TCP);
-        cfg.setPort(port);
-
-        return cfg;
-    }
-
-    /**
      * Start grid with IGFS.
      *
      * @param gridName Grid name.
@@ -296,8 +282,7 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
         final AtomicBoolean stop = new AtomicBoolean();
 
         GridTestUtils.runMultiThreadedAsync(new Callable() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 Thread.sleep(1_000); // Some delay to ensure read is in progress.
 
                 // Now stop all the nodes but the 1st:
@@ -390,7 +375,7 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
 
             final int f = f0;
 
-            int att = doWithRetries(2, new Callable<Void>() {
+            int att = doWithRetries(1, new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     IgfsOutputStream ios = os;
 
@@ -411,6 +396,8 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
                 }
             });
 
+            assert att == 1;
+
             X.println("write #2 completed: " + f0 + " in " + att + " attempts.");
         }
 
@@ -432,6 +419,120 @@ public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     *
+     * @throws Exception
+     */
+    public void testWriteFailoverWhileStoppingMultipleNodes() throws Exception {
+        final IgfsImpl igfs0 = nodeDatas[0].igfsImpl;
+
+        clear(igfs0);
+
+        IgfsAbstractSelfTest.create(igfs0, paths(DIR, SUBDIR), null);
+
+        final IgfsOutputStream[] outStreams = new IgfsOutputStream[files];
+
+        // Create files:
+        for (int f = 0; f < files; f++) {
+            final byte[] data = createChunk(fileSize, f);
+
+            IgfsOutputStream os = null;
+
+            try {
+                os = igfs0.create(filePath(f), 256, true, null, 0, -1, null);
+
+                assert os != null;
+
+                writeFileChunks(os, data);
+            }
+            finally {
+                if (os != null)
+                    os.flush();
+            }
+
+            outStreams[f] = os;
+
+            X.println("write #1 completed: " + f);
+        }
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        GridTestUtils.runMultiThreadedAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                Thread.sleep(10_000); // Some delay to ensure read is in progress.
+
+                // Now stop all the nodes but the 1st:
+                for (int n = 1; n < numIgfsNodes; n++) {
+                    stopGrid(n);
+
+                    X.println("#### grid " + n + " stopped.");
+                }
+
+                //Thread.sleep(10_000);
+
+                stop.set(true);
+
+                return null;
+            }
+        }, 1, "igfs-node-stopper");
+
+        // Write #2:
+        for (int f0 = 0; f0 < files; f0++) {
+            final IgfsOutputStream os = outStreams[f0];
+
+            assert os != null;
+
+            final int f = f0;
+
+            int att = doWithRetries(1, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgfsOutputStream ios = os;
+
+                    try {
+                        writeChunks0(igfs0, ios, f);
+                    }
+                    catch (IOException ioe) {
+                        log().warning("Attempt to append the data to existing stream failed: ", ioe);
+
+                        ios = igfs0.append(filePath(f), false);
+
+                        assert ios != null;
+
+                        writeChunks0(igfs0, ios, f);
+                    }
+
+                    return null;
+                }
+            });
+
+            assert att == 1;
+
+            X.println("write #2 completed: " + f0 + " in " + att + " attempts.");
+        }
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return stop.get();
+            }
+        }, 25_000);
+
+        // Check files:
+        for (int f = 0; f < files; f++) {
+            IgfsPath path = filePath(f);
+
+            byte[] data = createChunk(fileSize, f);
+
+            // Check through 1st node:
+            checkExist(igfs0, path);
+
+            assertEquals("File length mismatch.", data.length * 2, igfs0.size(path));
+
+            checkFileContent(igfs0, path, data, data);
+
+            X.println("Read test completed: " + f);
+        }
+    }
+
+    /**
      * Writes data to the file of the specified index and closes the output stream.
      *
      * @param igfs0 IGFS.


[30/50] [abbrv] 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/3d46b624
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3d46b624
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3d46b624

Branch: refs/heads/ignite-843
Commit: 3d46b6248712c67fa1ff695b0a21e33bd9425691
Parents: 712b29c f497e8e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 13:29:31 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 13:29:31 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      | 115 +++++++++++--------
 1 file changed, 67 insertions(+), 48 deletions(-)
----------------------------------------------------------------------



[50/50] [abbrv] 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/331950c7
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/331950c7
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/331950c7

Branch: refs/heads/ignite-843
Commit: 331950c75d3e934846a8d24ce199896b51b4e91c
Parents: 231a88c 9f7dc50
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Mon Aug 31 08:59:59 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Aug 31 08:59:59 2015 +0700

----------------------------------------------------------------------
 .../ignite/internal/IgniteInternalFuture.java   |   10 +
 .../org/apache/ignite/internal/IgnitionEx.java  |   23 +-
 .../internal/portable/PortableContext.java      |  128 ++-
 .../processors/cache/GridCacheAdapter.java      |   10 +-
 .../processors/cache/GridCacheIoManager.java    |   10 +-
 .../cache/GridCacheSharedContext.java           |   14 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   10 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   87 +-
 .../CacheObjectPortableProcessorImpl.java       |    3 +-
 .../query/GridCacheQueryFutureAdapter.java      |    7 +
 .../cache/transactions/IgniteTxAdapter.java     |    1 +
 .../cacheobject/IgniteCacheObjectProcessor.java |    2 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../processors/igfs/IgfsDataManager.java        |  169 ++-
 .../processors/igfs/IgfsMetaManager.java        |   43 +-
 .../processors/igfs/IgfsOutputStreamImpl.java   |    2 +
 .../internal/processors/igfs/IgfsUtils.java     |   52 +
 .../platform/PlatformAwareEventFilter.java      |    4 +-
 .../processors/platform/PlatformContext.java    |  272 +++++
 .../platform/PlatformNoopProcessor.java         |   10 +
 .../processors/platform/PlatformProcessor.java  |   14 +
 .../processors/platform/PlatformTarget.java     |   76 ++
 .../cache/PlatformCacheEntryFilter.java         |   29 +
 .../cache/query/PlatformContinuousQuery.java    |   58 +
 .../callback/PlatformCallbackGateway.java       |  869 ++++++++++++++
 .../callback/PlatformCallbackUtils.java         |  468 ++++++++
 .../platform/compute/PlatformJob.java           |   39 +
 .../platform/memory/PlatformInputStream.java    |   30 +
 .../platform/memory/PlatformMemory.java         |   77 ++
 .../platform/memory/PlatformMemoryManager.java  |   46 +
 .../platform/memory/PlatformOutputStream.java   |   30 +
 .../platform/services/PlatformService.java      |   44 +
 .../processors/query/GridQueryProcessor.java    |    2 +-
 .../util/future/GridEmbeddedFuture.java         |   55 +-
 .../util/future/GridFinishedFuture.java         |    5 +
 .../internal/util/future/GridFutureAdapter.java |   58 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |    2 +-
 ...acheAsyncOperationsFailoverAbstractTest.java |  362 ++++++
 .../CacheAsyncOperationsFailoverAtomicTest.java |   32 +
 .../CacheAsyncOperationsFailoverTxTest.java     |   32 +
 .../distributed/CacheAsyncOperationsTest.java   |  280 +++++
 .../igfs/IgfsBackupFailoverSelfTest.java        |  137 ++-
 .../testframework/junits/GridAbstractTest.java  |    2 +-
 .../IgniteCacheFailoverTestSuite2.java          |    4 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    1 +
 .../testsuites/IgniteCacheTestSuite3.java       |    2 +
 .../Apache.Ignite.Core.csproj                   |   22 +
 .../Common/AsyncSupportedAttribute.cs           |   33 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |   52 +
 .../dotnet/Apache.Ignite.Core/Common/IFuture.cs |  115 ++
 .../Common/IgniteException.cs                   |   66 ++
 .../Impl/Collections/CollectionExtensions.cs    |   45 +
 .../Impl/Collections/MultiValueDictionary.cs    |  143 +++
 .../Impl/Collections/ReadOnlyCollection.cs      |  102 ++
 .../Impl/Collections/ReadOnlyDictionary.cs      |  149 +++
 .../Impl/Common/AsyncResult.cs                  |   71 ++
 .../Impl/Common/CompletedAsyncResult.cs         |   70 ++
 .../Common/CopyOnWriteConcurrentDictionary.cs   |   70 ++
 .../Impl/Common/DelegateConverter.cs            |  253 +++++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  286 +++++
 .../Impl/Common/FutureType.cs                   |   52 +
 .../Impl/Common/GridArgumentCheck.cs            |   76 ++
 .../Impl/Common/IFutureConverter.cs             |   32 +
 .../Impl/Common/IFutureInternal.cs              |   45 +
 .../Impl/Common/LoadedAssembliesResolver.cs     |   96 ++
 .../Impl/Common/TypeCaster.cs                   |   72 ++
 .../platform/PlatformAbstractPredicate.java     |   64 ++
 .../platform/PlatformAbstractTarget.java        |   44 +-
 .../processors/platform/PlatformBootstrap.java  |    1 -
 .../processors/platform/PlatformContext.java    |  114 --
 .../processors/platform/PlatformTarget.java     |   76 --
 .../platform/cache/PlatformCache.java           | 1056 ++++++++++++++++++
 .../cache/PlatformCacheEntryFilterImpl.java     |  105 ++
 .../cache/PlatformCacheEntryProcessor.java      |  212 ++++
 .../platform/cache/PlatformCacheIterator.java   |   72 ++
 .../PlatformCachePartialUpdateException.java    |   68 ++
 .../cache/affinity/PlatformAffinity.java        |  293 +++++
 .../query/PlatformAbstractQueryCursor.java      |  192 ++++
 .../query/PlatformContinuousQueryImpl.java      |  230 ++++
 .../PlatformContinuousQueryRemoteFilter.java    |  183 +++
 .../cache/query/PlatformFieldsQueryCursor.java  |   50 +
 .../cache/query/PlatformQueryCursor.java        |   46 +
 .../cache/store/PlatformCacheStore.java         |   25 +
 .../cache/store/PlatformCacheStoreCallback.java |   61 +
 .../callback/PlatformCallbackGateway.java       |  869 --------------
 .../callback/PlatformCallbackUtils.java         |  468 --------
 .../platform/cluster/PlatformClusterGroup.java  |  330 ++++++
 .../cluster/PlatformClusterNodeFilter.java      |   77 ++
 .../platform/compute/PlatformAbstractJob.java   |  154 +++
 .../platform/compute/PlatformAbstractTask.java  |  202 ++++
 .../PlatformBalancingMultiClosureTask.java      |   78 ++
 ...tformBalancingSingleClosureAffinityTask.java |   86 ++
 .../PlatformBalancingSingleClosureTask.java     |   77 ++
 .../PlatformBroadcastingMultiClosureTask.java   |   83 ++
 .../PlatformBroadcastingSingleClosureTask.java  |   81 ++
 .../platform/compute/PlatformClosureJob.java    |  102 ++
 .../platform/compute/PlatformCompute.java       |  323 ++++++
 .../platform/compute/PlatformFullJob.java       |  217 ++++
 .../platform/compute/PlatformFullTask.java      |  185 +++
 .../compute/PlatformNativeException.java        |   75 ++
 .../datastreamer/PlatformDataStreamer.java      |  222 ++++
 .../datastreamer/PlatformStreamReceiver.java    |  114 ++
 .../platform/dotnet/PlatformDotNetService.java  |   27 +
 .../dotnet/PlatformDotNetServiceImpl.java       |   47 +
 .../platform/events/PlatformEventFilter.java    |  161 +++
 .../platform/events/PlatformEvents.java         |  388 +++++++
 .../platform/memory/PlatformInputStream.java    |   30 -
 .../platform/memory/PlatformMemory.java         |   77 --
 .../platform/memory/PlatformMemoryManager.java  |   46 -
 .../platform/memory/PlatformOutputStream.java   |   30 -
 .../messaging/PlatformMessageFilter.java        |  109 ++
 .../messaging/PlatformMessageLocalFilter.java   |  102 ++
 .../platform/messaging/PlatformMessaging.java   |  162 +++
 .../services/PlatformAbstractService.java       |  223 ++++
 .../platform/services/PlatformServices.java     |  252 +++++
 .../transactions/PlatformTransactions.java      |  255 +++++
 .../platform/utils/PlatformFutureUtils.java     |   12 +-
 .../platform/utils/PlatformUtils.java           |  157 ++-
 scripts/apply-pull-request.sh                   |    2 +
 120 files changed, 12176 insertions(+), 2008 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/331950c7/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/331950c7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------


[05/50] [abbrv] ignite git commit: IGNITE-1309: Moved platform affinity to Ignite.

Posted by ak...@apache.org.
IGNITE-1309: Moved platform affinity to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 536af49ba0a407fdbc5682f73a48aa07fa3daae0
Parents: d9a1397
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 17:03:39 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 17:03:39 2015 +0300

----------------------------------------------------------------------
 .../cache/affinity/PlatformAffinity.java        | 293 +++++++++++++++++++
 1 file changed, 293 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/536af49b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
new file mode 100644
index 0000000..d6dfcdb
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
@@ -0,0 +1,293 @@
+/*
+ * 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.affinity;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Native cache wrapper implementation.
+ */
+@SuppressWarnings({"unchecked", "UnusedDeclaration", "TryFinallyCanBeTryWithResources"})
+public class PlatformAffinity extends PlatformAbstractTarget {
+    /** */
+    public static final int OP_AFFINITY_KEY = 1;
+
+    /** */
+    public static final int OP_ALL_PARTITIONS = 2;
+
+    /** */
+    public static final int OP_BACKUP_PARTITIONS = 3;
+
+    /** */
+    public static final int OP_IS_BACKUP = 4;
+
+    /** */
+    public static final int OP_IS_PRIMARY = 5;
+
+    /** */
+    public static final int OP_IS_PRIMARY_OR_BACKUP = 6;
+
+    /** */
+    public static final int OP_MAP_KEY_TO_NODE = 7;
+
+    /** */
+    public static final int OP_MAP_KEY_TO_PRIMARY_AND_BACKUPS = 8;
+
+    /** */
+    public static final int OP_MAP_KEYS_TO_NODES = 9;
+
+    /** */
+    public static final int OP_MAP_PARTITION_TO_NODE = 10;
+
+    /** */
+    public static final int OP_MAP_PARTITION_TO_PRIMARY_AND_BACKUPS = 11;
+
+    /** */
+    public static final int OP_MAP_PARTITIONS_TO_NODES = 12;
+
+    /** */
+    public static final int OP_PARTITION = 13;
+
+    /** */
+    public static final int OP_PRIMARY_PARTITIONS = 14;
+
+    /** */
+    private static final C1<ClusterNode, UUID> TO_NODE_ID = new C1<ClusterNode, UUID>() {
+        @Nullable @Override public UUID apply(ClusterNode node) {
+            return node != null ? node.id() : null;
+        }
+    };
+
+    /** Underlying cache affinity. */
+    private final Affinity<Object> aff;
+
+    /** Discovery manager */
+    private final GridDiscoveryManager discovery;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param igniteCtx Ignite context.
+     * @param name Cache name.
+     */
+    public PlatformAffinity(PlatformContext platformCtx, GridKernalContext igniteCtx, @Nullable String name)
+        throws IgniteCheckedException {
+        super(platformCtx);
+
+        this.aff = igniteCtx.grid().affinity(name);
+
+        if (aff == null)
+            throw new IgniteCheckedException("Cache with the given name doesn't exist: " + name);
+
+        discovery = igniteCtx.discovery();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_PARTITION:
+                return aff.partition(reader.readObjectDetached());
+
+            case OP_IS_PRIMARY: {
+                UUID nodeId = reader.readUuid();
+
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                if (node == null)
+                    return FALSE;
+
+                return aff.isPrimary(node, key) ? TRUE : FALSE;
+            }
+
+            case OP_IS_BACKUP: {
+                UUID nodeId = reader.readUuid();
+
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                if (node == null)
+                    return FALSE;
+
+                return aff.isBackup(node, key) ? TRUE : FALSE;
+            }
+
+            case OP_IS_PRIMARY_OR_BACKUP: {
+                UUID nodeId = reader.readUuid();
+
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                if (node == null)
+                    return FALSE;
+
+                return aff.isPrimaryOrBackup(node, key) ? TRUE : FALSE;
+            }
+
+            default:
+                return throwUnsupported(type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_PRIMARY_PARTITIONS: {
+                UUID nodeId = reader.readObject();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                int[] parts = node != null ? aff.primaryPartitions(node) : U.EMPTY_INTS;
+
+                writer.writeIntArray(parts);
+
+                break;
+            }
+
+            case OP_BACKUP_PARTITIONS: {
+                UUID nodeId = reader.readObject();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                int[] parts = node != null ? aff.backupPartitions(node) : U.EMPTY_INTS;
+
+                writer.writeIntArray(parts);
+
+                break;
+            }
+
+            case OP_ALL_PARTITIONS: {
+                UUID nodeId = reader.readObject();
+
+                ClusterNode node = discovery.node(nodeId);
+
+                int[] parts = node != null ? aff.allPartitions(node) : U.EMPTY_INTS;
+
+                writer.writeIntArray(parts);
+
+                break;
+            }
+
+            case OP_AFFINITY_KEY: {
+                Object key = reader.readObjectDetached();
+
+                writer.writeObject(aff.affinityKey(key));
+
+                break;
+            }
+
+            case OP_MAP_KEY_TO_NODE: {
+                Object key = reader.readObjectDetached();
+
+                ClusterNode node = aff.mapKeyToNode(key);
+
+                platformCtx.writeNode(writer, node);
+
+                break;
+            }
+
+            case OP_MAP_PARTITION_TO_NODE: {
+                int part = reader.readObject();
+
+                ClusterNode node = aff.mapPartitionToNode(part);
+
+                platformCtx.writeNode(writer, node);
+
+                break;
+            }
+
+            case OP_MAP_KEY_TO_PRIMARY_AND_BACKUPS: {
+                Object key = reader.readObjectDetached();
+
+                platformCtx.writeNodes(writer, aff.mapKeyToPrimaryAndBackups(key));
+
+                break;
+            }
+
+            case OP_MAP_PARTITION_TO_PRIMARY_AND_BACKUPS: {
+                int part = reader.readObject();
+
+                platformCtx.writeNodes(writer, aff.mapPartitionToPrimaryAndBackups(part));
+
+                break;
+            }
+
+            case OP_MAP_KEYS_TO_NODES: {
+                Collection<Object> keys = reader.readCollection();
+
+                Map<ClusterNode, Collection<Object>> map = aff.mapKeysToNodes(keys);
+
+                writer.writeInt(map.size());
+
+                for (Map.Entry<ClusterNode, Collection<Object>> e : map.entrySet()) {
+                    platformCtx.addNode(e.getKey());
+
+                    writer.writeUuid(e.getKey().id());
+                    writer.writeObject(e.getValue());
+                }
+
+                break;
+            }
+
+            case OP_MAP_PARTITIONS_TO_NODES: {
+                Collection<Integer> parts = reader.readCollection();
+
+                Map<Integer, ClusterNode> map = aff.mapPartitionsToNodes(parts);
+
+                writer.writeInt(map.size());
+
+                for (Map.Entry<Integer, ClusterNode> e : map.entrySet()) {
+                    platformCtx.addNode(e.getValue());
+
+                    writer.writeInt(e.getKey());
+
+                    writer.writeUuid(e.getValue().id());
+                }
+
+                break;
+            }
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /**
+     * @return Gets number of partitions in cache.
+     */
+    public int partitions() {
+        return aff.partitions();
+    }
+}


[20/50] [abbrv] ignite git commit: Moved platform abstract predicate to Ignite.

Posted by ak...@apache.org.
Moved platform abstract predicate to Ignite.


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

Branch: refs/heads/ignite-843
Commit: daa87962306aa86f385c825d047febcfb3635aed
Parents: 6b2ee50
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 12:16:25 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 12:16:25 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractPredicate.java     | 64 ++++++++++++++++++++
 1 file changed, 64 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/daa87962/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractPredicate.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractPredicate.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractPredicate.java
new file mode 100644
index 0000000..c5197fd
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractPredicate.java
@@ -0,0 +1,64 @@
+/*
+ * 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.io.*;
+
+/**
+ * Base interop predicate. Delegates apply to native platform.
+ */
+public abstract class PlatformAbstractPredicate implements Externalizable {
+    /** .Net portable predicate */
+    protected Object pred;
+
+    /** Pointer to deployed predicate. */
+    protected transient long ptr;
+
+    /** Interop processor. */
+    protected transient PlatformContext ctx;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    public PlatformAbstractPredicate() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param pred .Net portable predicate.
+     * @param ptr Pointer to predicate in the native platform.
+     * @param ctx Kernal context.
+     */
+    protected PlatformAbstractPredicate(Object pred, long ptr, PlatformContext ctx) {
+        this.pred = pred;
+        this.ptr = ptr;
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(pred);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        pred = in.readObject();
+    }
+}


[16/50] [abbrv] ignite git commit: ignite-1217: add suffic to comment " - Fixes #."

Posted by ak...@apache.org.
ignite-1217: add suffic to comment " - Fixes #<id>."


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

Branch: refs/heads/ignite-843
Commit: cfc4d6615234f3f113193f43f8fc3c6c659ef680
Parents: e4ba2eb
Author: ashutak <as...@gridgain.com>
Authored: Wed Aug 26 22:27:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Aug 27 12:08:58 2015 +0300

----------------------------------------------------------------------
 scripts/apply-pull-request.sh | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cfc4d661/scripts/apply-pull-request.sh
----------------------------------------------------------------------
diff --git a/scripts/apply-pull-request.sh b/scripts/apply-pull-request.sh
index baa73b5..d852d78 100755
--- a/scripts/apply-pull-request.sh
+++ b/scripts/apply-pull-request.sh
@@ -128,6 +128,8 @@ if [ "${COMMENT}" == "" ]; then
     COMMENT=${ORIG_COMMENT}
 fi
 
+COMMENT="${COMMENT} - Fixes #${PR_ID}."
+
 git commit --author "${AUTHOR}" -a -s -m "${COMMENT}" &> /dev/null
 
 echo "Squash commit for pull request with id='${PR_ID}' has been added. The commit has been added with comment '${COMMENT}'."


[27/50] [abbrv] 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/7c2c02bf
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7c2c02bf
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7c2c02bf

Branch: refs/heads/ignite-843
Commit: 7c2c02bf0fb57e90c22bb28e71796b378f1d755b
Parents: a312934 00b27ce
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 13:17:43 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 13:17:43 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   6 +-
 .../util/future/GridEmbeddedFuture.java         |  55 +++-
 .../distributed/CacheAsyncOperationsTest.java   | 280 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite3.java       |   2 +
 5 files changed, 342 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[19/50] [abbrv] 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/a2b7ba1b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a2b7ba1b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a2b7ba1b

Branch: refs/heads/ignite-843
Commit: a2b7ba1bb70ca67d09479b9a4ca1ab4d87cced08
Parents: cfc4d66 6b2ee50
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Aug 27 12:12:54 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Aug 27 12:12:54 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteInternalFuture.java   | 10 ++++
 .../org/apache/ignite/internal/IgnitionEx.java  | 23 +++++++-
 .../query/GridCacheQueryFutureAdapter.java      |  7 +++
 .../processors/igfs/IgfsMetaManager.java        | 43 +++++++--------
 .../internal/processors/igfs/IgfsUtils.java     | 52 ++++++++++++++++++
 .../util/future/GridFinishedFuture.java         |  5 ++
 .../internal/util/future/GridFutureAdapter.java | 58 +++++++++++++-------
 .../testframework/junits/GridAbstractTest.java  |  2 +-
 8 files changed, 153 insertions(+), 47 deletions(-)
----------------------------------------------------------------------



[45/50] [abbrv] ignite git commit: IGNITE-1318: Moved platform data streamer to Ignite.

Posted by ak...@apache.org.
IGNITE-1318: Moved platform data streamer to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 16c095a9e9a30db630caa8a6ecec98ac5256962e
Parents: 207b682
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 15:01:06 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 15:01:06 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |  11 +
 .../datastreamer/PlatformDataStreamer.java      | 222 +++++++++++++++++++
 .../datastreamer/PlatformStreamReceiver.java    | 114 ++++++++++
 3 files changed, 347 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16c095a9/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 cbcc91b..9b4a891 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
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.platform.cache.query.*;
 import org.apache.ignite.internal.processors.platform.callback.*;
 import org.apache.ignite.internal.processors.platform.compute.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.stream.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -249,4 +250,14 @@ public interface PlatformContext {
      * @return Entry filter.
      */
     public PlatformCacheEntryFilter createCacheEntryFilter(Object filter, long ptr);
+
+    /**
+     * Create stream receiver.
+     *
+     * @param rcv Native receiver.
+     * @param ptr Pointer.
+     * @param keepPortable Keep portable flag.
+     * @return Stream receiver.
+     */
+    public StreamReceiver createStreamReceiver(Object rcv, long ptr, boolean keepPortable);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/16c095a9/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
new file mode 100644
index 0000000..fc9f535
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -0,0 +1,222 @@
+/*
+ * 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.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.managers.eventstorage.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.datastreamer.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.lang.*;
+
+import java.util.*;
+
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ * Interop data streamer wrapper.
+ */
+@SuppressWarnings({"UnusedDeclaration", "unchecked"})
+public class PlatformDataStreamer extends PlatformAbstractTarget {
+    /** Policy: continue. */
+    private static final int PLC_CONTINUE = 0;
+
+    /** Policy: close. */
+    private static final int PLC_CLOSE = 1;
+
+    /** Policy: cancel and close. */
+    private static final int PLC_CANCEL_CLOSE = 2;
+
+    /** Policy: do flush. */
+    private static final int PLC_FLUSH = 3;
+
+    /** */
+    private static final int OP_UPDATE = 1;
+
+    /** */
+    private static final int OP_RECEIVER = 2;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Data streamer. */
+    private final DataStreamerImpl ldr;
+
+    /** Portable flag. */
+    private final boolean keepPortable;
+
+    /** Topology update event listener. */
+    private volatile GridLocalEventListener lsnr;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param ldr Data streamer.
+     */
+    public PlatformDataStreamer(PlatformContext platformCtx, String cacheName, DataStreamerImpl ldr,
+        boolean keepPortable) {
+        super(platformCtx);
+
+        this.cacheName = cacheName;
+        this.ldr = ldr;
+        this.keepPortable = keepPortable;
+    }
+
+    /** {@inheritDoc}  */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_UPDATE:
+                int plc = reader.readInt();
+
+                if (plc == PLC_CANCEL_CLOSE) {
+                    // Close with cancel.
+                    platformCtx.kernalContext().event().removeLocalEventListener(lsnr);
+
+                    ldr.close(true);
+                }
+                else {
+                    final long futPtr = reader.readLong();
+
+                    int valsCnt = reader.readInt();
+
+                    if (valsCnt > 0) {
+                        Collection<GridMapEntry> vals = new ArrayList<>(valsCnt);
+
+                        for (int i = 0; i < valsCnt; i++)
+                            vals.add(new GridMapEntry(reader.readObjectDetached(), reader.readObjectDetached()));
+
+                        PlatformFutureUtils.listen(platformCtx, ldr.addData(vals), futPtr,
+                            PlatformFutureUtils.TYP_OBJ);
+                    }
+
+                    if (plc == PLC_CLOSE) {
+                        platformCtx.kernalContext().event().removeLocalEventListener(lsnr);
+
+                        ldr.close(false);
+                    }
+                    else if (plc == PLC_FLUSH)
+                        ldr.tryFlush();
+                    else
+                        assert plc == PLC_CONTINUE;
+                }
+
+                return TRUE;
+
+            case OP_RECEIVER:
+                long ptr = reader.readLong();
+
+                Object rec = reader.readObjectDetached();
+
+                ldr.receiver(platformCtx.createStreamReceiver(rec, ptr, keepPortable));
+
+                return TRUE;
+
+            default:
+                return throwUnsupported(type);
+        }
+    }
+
+    /**
+     * Listen topology changes.
+     *
+     * @param ptr Pointer.
+     */
+    public void listenTopology(final long ptr) {
+        lsnr = new GridLocalEventListener() {
+            @Override public void onEvent(Event evt) {
+                DiscoveryEvent discoEvt = (DiscoveryEvent)evt;
+
+                long topVer = discoEvt.topologyVersion();
+                int topSize = platformCtx.kernalContext().discovery().cacheNodes(
+                    cacheName, new AffinityTopologyVersion(topVer)).size();
+
+                platformCtx.gateway().dataStreamerTopologyUpdate(ptr, topVer, topSize);
+            }
+        };
+
+        platformCtx.kernalContext().event().addLocalEventListener(lsnr, EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT);
+
+        GridDiscoveryManager discoMgr = platformCtx.kernalContext().discovery();
+
+        long topVer = discoMgr.topologyVersion();
+        int topSize = discoMgr.cacheNodes(cacheName, new AffinityTopologyVersion(topVer)).size();
+
+        platformCtx.gateway().dataStreamerTopologyUpdate(ptr, topVer, topSize);
+    }
+
+    /**
+     * @return Allow-overwrite flag.
+     */
+    public boolean allowOverwrite() {
+        return ldr.allowOverwrite();
+    }
+
+    /**
+     * @param val Allow-overwrite flag.
+     */
+    public void allowOverwrite(boolean val) {
+        ldr.allowOverwrite(val);
+    }
+
+    /**
+     * @return Skip store flag.
+     */
+    public boolean skipStore() {
+        return ldr.skipStore();
+    }
+
+    /**
+     * @param skipStore Skip store flag.
+     */
+    public void skipStore(boolean skipStore) {
+        ldr.skipStore(skipStore);
+    }
+
+    /**
+     * @return Per-node buffer size.
+     */
+    public int perNodeBufferSize() {
+        return ldr.perNodeBufferSize();
+    }
+
+    /**
+     * @param val Per-node buffer size.
+     */
+    public void perNodeBufferSize(int val) {
+        ldr.perNodeBufferSize(val);
+    }
+
+    /**
+     * @return Per-node parallel load operations.
+     */
+    public int perNodeParallelOperations() {
+        return ldr.perNodeParallelOperations();
+    }
+
+    /**
+     * @param val Per-node parallel load operations.
+     */
+    public void perNodeParallelOperations(int val) {
+        ldr.perNodeParallelOperations(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/16c095a9/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
new file mode 100644
index 0000000..70bfb6b
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiver.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.datastreamer;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.cache.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.stream.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * 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();
+    }
+
+}


[13/50] [abbrv] ignite git commit: WIP on cluster node and cluster metrics.

Posted by ak...@apache.org.
WIP on cluster node and cluster metrics.


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

Branch: refs/heads/ignite-843
Commit: e4ba2ebaf2544ac978c201a6a02e6330e1d5ac81
Parents: 5718480
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 11:28:46 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 11:28:46 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/portable/PortableContext.java    | 3 ---
 .../internal/processors/platform/PlatformContext.java       | 9 +++++++++
 2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e4ba2eba/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 2d3cbf0..a9d64d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -204,15 +204,12 @@ public class PortableContext implements Externalizable {
 //        registerPredefinedType(DrSenderAttributes.class, 65);
 //        registerPredefinedType(DrSenderRemoteAttributes.class, 66);
 //
-//        registerPredefinedType(InteropClusterNode.class, 67);
-//        registerPredefinedType(InteropClusterMetrics.class, 68);
 //        registerPredefinedType(InteropMetadata.class, 70);
 //
 //        registerPredefinedType(InteropDotNetConfiguration.class, 71);
 //        registerPredefinedType(InteropDotNetPortableConfiguration.class, 72);
 //        registerPredefinedType(InteropDotNetPortableTypeConfiguration.class, 73);
 //        registerPredefinedType(InteropIgniteProxy.class, 74);
-//        registerPredefinedType(InteropCacheMetrics.class, 75);
 //        registerPredefinedType(InteropProductLicence.class, 78);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4ba2eba/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 fb1eaa2..90ed85d 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
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.portable.*;
 import org.apache.ignite.internal.processors.platform.callback.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -111,4 +112,12 @@ public interface PlatformContext {
      * @param reader Reader.
      */
     public void processMetadata(PortableRawReaderEx reader);
+
+    /**
+     * Write cluster metrics.
+     *
+     * @param writer Writer.
+     * @param metrics Metrics.
+     */
+    public void writeClusterMetrics(PortableRawWriterEx writer, @Nullable ClusterMetrics metrics);
 }


[26/50] [abbrv] ignite git commit: ignite-1300: add ability to register class types outside of the portable context

Posted by ak...@apache.org.
ignite-1300: add ability to register class types outside of the portable context


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

Branch: refs/heads/ignite-843
Commit: f575ff119d589c04836a3de065e5f958daec8000
Parents: 00b27ce
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Aug 27 13:17:35 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Aug 27 13:17:35 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      | 115 +++++++++++--------
 1 file changed, 67 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f575ff11/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index a9d64d9..723113e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -114,10 +114,19 @@ public class PortableContext implements Externalizable {
     private String gridName;
 
     /** */
-    private PortableMarshaller marsh;
+    private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller();
 
     /** */
-    private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller();
+    private boolean convertStrings;
+
+    /** */
+    private boolean useTs;
+
+    /** */
+    private boolean metaDataEnabled;
+
+    /** */
+    private boolean keepDeserialized;
 
     /**
      * For {@link Externalizable}.
@@ -199,18 +208,6 @@ public class PortableContext implements Externalizable {
         registerPredefinedType(PortableObjectImpl.class, 63);
 
         registerPredefinedType(PortableMetaDataImpl.class, 64);
-
-// TODO: IGNITE-1258
-//        registerPredefinedType(DrSenderAttributes.class, 65);
-//        registerPredefinedType(DrSenderRemoteAttributes.class, 66);
-//
-//        registerPredefinedType(InteropMetadata.class, 70);
-//
-//        registerPredefinedType(InteropDotNetConfiguration.class, 71);
-//        registerPredefinedType(InteropDotNetPortableConfiguration.class, 72);
-//        registerPredefinedType(InteropDotNetPortableTypeConfiguration.class, 73);
-//        registerPredefinedType(InteropIgniteProxy.class, 74);
-//        registerPredefinedType(InteropProductLicence.class, 78);
     }
 
     /**
@@ -221,40 +218,68 @@ public class PortableContext implements Externalizable {
         if (marsh == null)
             return;
 
-        this.marsh = marsh;
+        convertStrings = marsh.isConvertStringToBytes();
+        useTs = marsh.isUseTimestamp();
+        metaDataEnabled = marsh.isMetaDataEnabled();
+        keepDeserialized = marsh.isKeepDeserialized();
+
         marshCtx = marsh.getContext();
 
         assert marshCtx != null;
 
         optmMarsh.setContext(marshCtx);
 
-        PortableIdMapper globalIdMapper = marsh.getIdMapper();
-        PortableSerializer globalSerializer = marsh.getSerializer();
-        boolean globalUseTs = marsh.isUseTimestamp();
-        boolean globalMetaDataEnabled = marsh.isMetaDataEnabled();
-        boolean globalKeepDeserialized = marsh.isKeepDeserialized();
+        configure(
+            marsh.getIdMapper(),
+            marsh.getSerializer(),
+            marsh.isUseTimestamp(),
+            marsh.isMetaDataEnabled(),
+            marsh.isKeepDeserialized(),
+            marsh.getClassNames(),
+            marsh.getTypeConfigurations()
+        );
+    }
 
+    /**
+     * @param globalIdMapper ID mapper.
+     * @param globalSerializer Serializer.
+     * @param globalUseTs Use timestamp flag.
+     * @param globalMetaDataEnabled Metadata enabled flag.
+     * @param globalKeepDeserialized Keep deserialized flag.
+     * @param clsNames Class names.
+     * @param typeCfgs Type configurations.
+     * @throws PortableException In case of error.
+     */
+    private void configure(
+        PortableIdMapper globalIdMapper,
+        PortableSerializer globalSerializer,
+        boolean globalUseTs,
+        boolean globalMetaDataEnabled,
+        boolean globalKeepDeserialized,
+        Collection<String> clsNames,
+        Collection<PortableTypeConfiguration> typeCfgs
+    ) throws PortableException {
         TypeDescriptors descs = new TypeDescriptors();
 
-        if (marsh.getClassNames() != null) {
+        if (clsNames != null) {
             PortableIdMapper idMapper = new IdMapperWrapper(globalIdMapper);
 
-            for (String clsName : marsh.getClassNames()) {
+            for (String clsName : clsNames) {
                 if (clsName.endsWith(".*")) { // Package wildcard
                     String pkgName = clsName.substring(0, clsName.length() - 2);
 
                     for (String clsName0 : classesInPackage(pkgName))
                         descs.add(clsName0, idMapper, null, null, globalUseTs, globalMetaDataEnabled,
-                                  globalKeepDeserialized, true);
+                            globalKeepDeserialized, true);
                 }
                 else // Regular single class
                     descs.add(clsName, idMapper, null, null, globalUseTs, globalMetaDataEnabled,
-                              globalKeepDeserialized, true);
+                        globalKeepDeserialized, true);
             }
         }
 
-        if (marsh.getTypeConfigurations() != null) {
-            for (PortableTypeConfiguration typeCfg : marsh.getTypeConfigurations()) {
+        if (typeCfgs != null) {
+            for (PortableTypeConfiguration typeCfg : typeCfgs) {
                 String clsName = typeCfg.getClassName();
 
                 if (clsName == null)
@@ -283,17 +308,18 @@ public class PortableContext implements Externalizable {
 
                     for (String clsName0 : classesInPackage(pkgName))
                         descs.add(clsName0, idMapper, serializer, typeCfg.getAffinityKeyFieldName(), useTs,
-                                  metaDataEnabled, keepDeserialized, true);
+                            metaDataEnabled, keepDeserialized, true);
                 }
                 else
                     descs.add(clsName, idMapper, serializer, typeCfg.getAffinityKeyFieldName(), useTs,
-                              metaDataEnabled, keepDeserialized, false);
+                        metaDataEnabled, keepDeserialized, false);
             }
         }
 
-        for (TypeDescriptor desc : descs.descriptors())
+        for (TypeDescriptor desc : descs.descriptors()) {
             registerUserType(desc.clsName, desc.idMapper, desc.serializer, desc.affKeyFieldName, desc.useTs,
-                             desc.metadataEnabled, desc.keepDeserialized);
+                desc.metadataEnabled, desc.keepDeserialized);
+        }
     }
 
     /**
@@ -437,9 +463,9 @@ public class PortableContext implements Externalizable {
                 clsName,
                 BASIC_CLS_ID_MAPPER,
                 null,
-                marsh.isUseTimestamp(),
-                marsh.isMetaDataEnabled(),
-                marsh.isKeepDeserialized());
+                useTs,
+                metaDataEnabled,
+                keepDeserialized);
 
             PortableClassDescriptor old = descByCls.putIfAbsent(cls, desc);
 
@@ -483,9 +509,9 @@ public class PortableContext implements Externalizable {
             typeName,
             idMapper,
             null,
-            marsh.isUseTimestamp(),
-            marsh.isMetaDataEnabled(),
-            marsh.isKeepDeserialized(),
+            useTs,
+            metaDataEnabled,
+            keepDeserialized,
             registered);
 
         // perform put() instead of putIfAbsent() because "registered" flag may have been changed.
@@ -632,9 +658,9 @@ public class PortableContext implements Externalizable {
     /**
      * @param cls Class.
      * @param id Type ID.
-     * @return PortableClassDescriptor.
+     * @return GridPortableClassDescriptor.
      */
-    private PortableClassDescriptor registerPredefinedType(Class<?> cls, int id) {
+    public PortableClassDescriptor registerPredefinedType(Class<?> cls, int id) {
         PortableClassDescriptor desc = new PortableClassDescriptor(
             this,
             cls,
@@ -734,13 +760,6 @@ public class PortableContext implements Externalizable {
     }
 
     /**
-     * @return Whether meta data is globally enabled.
-     */
-    boolean isMetaDataEnabled() {
-        return marsh.isMetaDataEnabled();
-    }
-
-    /**
      * @param typeId Type ID.
      * @return Whether meta data is enabled.
      */
@@ -794,7 +813,7 @@ public class PortableContext implements Externalizable {
      * @return Use timestamp flag.
      */
     public boolean isUseTimestamp() {
-        return marsh.isUseTimestamp();
+        return useTs;
     }
 
     /**
@@ -809,7 +828,7 @@ public class PortableContext implements Externalizable {
      * @return Whether to convert string to UTF8 bytes.
      */
     public boolean isConvertString() {
-        return marsh.isConvertStringToBytes();
+        return convertStrings;
     }
 
     /**
@@ -1082,4 +1101,4 @@ public class PortableContext implements Externalizable {
             return registered;
         }
     }
-}
+}
\ No newline at end of file


[40/50] [abbrv] ignite git commit: 1.3.3-p6-SNAPSHOT

Posted by ak...@apache.org.
1.3.3-p6-SNAPSHOT


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

Branch: refs/heads/ignite-843
Commit: 612a6b8142b7296051bc3702028fcead109bad9c
Parents: 56fd272
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Aug 28 11:47:01 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Aug 28 11:47:01 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                   | 2 +-
 modules/aop/pom.xml                                | 2 +-
 modules/apache-license-gen/pom.xml                 | 2 +-
 modules/aws/pom.xml                                | 2 +-
 modules/clients/pom.xml                            | 2 +-
 modules/cloud/pom.xml                              | 2 +-
 modules/codegen/pom.xml                            | 2 +-
 modules/core/pom.xml                               | 2 +-
 modules/core/src/main/resources/ignite.properties  | 2 +-
 modules/extdata/p2p/pom.xml                        | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml | 2 +-
 modules/extdata/uri/pom.xml                        | 2 +-
 modules/gce/pom.xml                                | 2 +-
 modules/geospatial/pom.xml                         | 2 +-
 modules/hadoop/pom.xml                             | 2 +-
 modules/hibernate/pom.xml                          | 2 +-
 modules/indexing/pom.xml                           | 2 +-
 modules/jcl/pom.xml                                | 2 +-
 modules/jta/pom.xml                                | 2 +-
 modules/kafka/pom.xml                              | 2 +-
 modules/log4j/pom.xml                              | 2 +-
 modules/mesos/pom.xml                              | 2 +-
 modules/rest-http/pom.xml                          | 2 +-
 modules/scalar-2.10/pom.xml                        | 2 +-
 modules/scalar/pom.xml                             | 2 +-
 modules/schedule/pom.xml                           | 2 +-
 modules/schema-import/pom.xml                      | 2 +-
 modules/slf4j/pom.xml                              | 2 +-
 modules/spark-2.10/pom.xml                         | 2 +-
 modules/spark/pom.xml                              | 2 +-
 modules/spring/pom.xml                             | 2 +-
 modules/ssh/pom.xml                                | 2 +-
 modules/tools/pom.xml                              | 2 +-
 modules/urideploy/pom.xml                          | 2 +-
 modules/visor-console-2.10/pom.xml                 | 2 +-
 modules/visor-console/pom.xml                      | 2 +-
 modules/visor-plugins/pom.xml                      | 2 +-
 modules/web/pom.xml                                | 2 +-
 modules/yardstick/pom.xml                          | 2 +-
 modules/yarn/pom.xml                               | 2 +-
 pom.xml                                            | 2 +-
 41 files changed, 41 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 0e9181c..c4445fa 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 8a6260b..13915f1 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 70cffe5..144cb45 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,5 +31,5 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index c86d3de..60aaa17 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index cd2e6e5..8f44d23 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index f2cd811..6204d35 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index b5c69eb..7ada8fc 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 40c01a2..1539550 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 4ea4173..8e38b9e 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.3.3-p5
+ignite.version=1.3.3-p6-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 5ff2ef2..97a3efd 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index 1b87e2e..aee39b4 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 86ced23..ad45998 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 7535aa9..c399c02 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index b50863c..d5d2b17 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 40b202a..2dfae51 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index dd1c2f5..7ed8bf7 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index c63eebe..b05c963 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 8a6e036..5b48182 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index f985a6b..52f7717 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 8583feb..691e445 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 3066d11..7b52dc0 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 69f3e74..cb40805 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <properties>
         <mesos.version>0.22.0</mesos.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 3763ba3..2ae943a 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index b1dfab6..86b80e6 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 07f6621..c853592 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 060c690..b748944 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index bbbaede..4b5ae9f 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 0c55dd8..e00a38a 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 087cac7..cae8086 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 27718f4..f61b3f9 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index efc95b2..4e38ddc 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 7f53ac0..b25822f 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index c5985ca..67940d6 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 072ad34..d08a5fb 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index 6fdef84..c43e452 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index c879e46..db33d7b 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 9b1c0f2..a1a67ea 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 2d54f11..1f930af 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 06ef51f..f48ce2e 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 946f282..ebac7b2 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/612a6b81/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1e7556d..642e344 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.3.3-p5-SNAPSHOT</version>
+    <version>1.3.3-p6-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[49/50] [abbrv] ignite git commit: Merge branch ignite-1.3.3-p6 into master.

Posted by ak...@apache.org.
Merge branch ignite-1.3.3-p6 into master.


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

Branch: refs/heads/ignite-843
Commit: 9f7dc5001a961a20e2bf24b3d6037585208ab2c1
Parents: 37a0505 612a6b8
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Aug 28 17:08:13 2015 -0700
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Aug 28 17:08:13 2015 -0700

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheIoManager.java | 10 +++++++++-
 .../processors/cache/GridCacheSharedContext.java      | 14 ++++++++------
 .../cache/transactions/IgniteTxAdapter.java           |  1 +
 3 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9f7dc500/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/9f7dc500/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index cc661d3,262d17a..10d2796
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@@ -329,14 -242,16 +329,16 @@@ public class GridCacheSharedContext<K, 
          if (preloadersStartFut == null) {
              GridCompoundFuture<Object, Object> compound = null;
  
 -            for (GridCacheContext cacheCtx : cacheContexts()) {
 +            for (GridCacheContext<?, ?> cacheCtx : cacheContexts()) {
-                 IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
+                 if (cacheCtx.startTopologyVersion() != null && cacheCtx.startTopologyVersion().compareTo(topVer) <= 0) {
+                     IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
  
-                 if (!startFut.isDone()) {
-                     if (compound == null)
-                         compound = new GridCompoundFuture<>();
+                     if (!startFut.isDone()) {
+                         if (compound == null)
+                             compound = new GridCompoundFuture<>();
  
-                     compound.add(startFut);
+                         compound.add(startFut);
+                     }
                  }
              }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/9f7dc500/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------


[10/50] [abbrv] ignite git commit: Further refactorings necessary for platforms move to Ignite.

Posted by ak...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
deleted file mode 100644
index 7e9587f..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
+++ /dev/null
@@ -1,468 +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.callback;
-
-/**
- * Platform callback utility methods. Implemented in target platform. All methods in this class must be
- * package-visible and invoked only through {@link PlatformCallbackGateway}.
- */
-public class PlatformCallbackUtils {
-    /**
-     * Create cache store.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    static native long cacheStoreCreate(long envPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Object pointer.
-     * @param memPtr Memory pointer.
-     * @param cb Callback.
-     * @return Result.
-     */
-    static native int cacheStoreInvoke(long envPtr, long objPtr, long memPtr, Object cb);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Object pointer.
-     */
-    static native void cacheStoreDestroy(long envPtr, long objPtr);
-
-    /**
-     * Creates cache store session.
-     *
-     * @param envPtr Environment pointer.
-     * @param storePtr Store instance pointer.
-     * @return Session instance pointer.
-     */
-    static native long cacheStoreSessionCreate(long envPtr, long storePtr);
-
-    /**
-     * Creates cache entry filter and returns a pointer.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    static native long cacheEntryFilterCreate(long envPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    static native int cacheEntryFilterApply(long envPtr, long objPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Pointer.
-     */
-    static native void cacheEntryFilterDestroy(long envPtr, long objPtr);
-
-    /**
-     * Invoke cache entry processor.
-     *
-     * @param envPtr Environment pointer.
-     * @param outMemPtr Output memory pointer.
-     * @param inMemPtr Input memory pointer.
-     */
-    static native void cacheInvoke(long envPtr, long outMemPtr, long inMemPtr);
-
-    /**
-     * Perform native task map. Do not throw exceptions, serializing them to the output stream instead.
-     *
-     * @param envPtr Environment pointer.
-     * @param taskPtr Task pointer.
-     * @param outMemPtr Output memory pointer (exists if topology changed, otherwise {@code 0}).
-     * @param inMemPtr Input memory pointer.
-     */
-    static native void computeTaskMap(long envPtr, long taskPtr, long outMemPtr, long inMemPtr);
-
-    /**
-     * Perform native task job result notification.
-     *
-     * @param envPtr Environment pointer.
-     * @param taskPtr Task pointer.
-     * @param jobPtr Job pointer.
-     * @param memPtr Memory pointer (always zero for local job execution).
-     * @return Job result enum ordinal.
-     */
-    static native int computeTaskJobResult(long envPtr, long taskPtr, long jobPtr, long memPtr);
-
-    /**
-     * Perform native task reduce.
-     *
-     * @param envPtr Environment pointer.
-     * @param taskPtr Task pointer.
-     */
-    static native void computeTaskReduce(long envPtr, long taskPtr);
-
-    /**
-     * Complete task with native error.
-     *
-     * @param envPtr Environment pointer.
-     * @param taskPtr Task pointer.
-     * @param memPtr Memory pointer with exception data or {@code 0} in case of success.
-     */
-    static native void computeTaskComplete(long envPtr, long taskPtr, long memPtr);
-
-    /**
-     * Serialize native job.
-     *
-     * @param envPtr Environment pointer.
-     * @param jobPtr Job pointer.
-     * @param memPtr Memory pointer.
-     * @return {@code True} if serialization succeeded.
-     */
-    static native int computeJobSerialize(long envPtr, long jobPtr, long memPtr);
-
-    /**
-     * Create job in native platform.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     * @return Pointer to job.
-     */
-    static native long computeJobCreate(long envPtr, long memPtr);
-
-    /**
-     * Execute native job on a node other than where it was created.
-     *
-     * @param envPtr Environment pointer.
-     * @param jobPtr Job pointer.
-     * @param cancel Cancel flag.
-     * @param memPtr Memory pointer to write result to for remote job execution or {@code 0} for local job execution.
-     */
-    static native void computeJobExecute(long envPtr, long jobPtr, int cancel, long memPtr);
-
-    /**
-     * Cancel the job.
-     *
-     * @param envPtr Environment pointer.
-     * @param jobPtr Job pointer.
-     */
-    static native void computeJobCancel(long envPtr, long jobPtr);
-
-    /**
-     * Destroy the job.
-     *
-     * @param envPtr Environment pointer.
-     * @param ptr Pointer.
-     */
-    static native void computeJobDestroy(long envPtr, long ptr);
-
-    /**
-     * Invoke local callback.
-     *
-     * @param envPtr Environment pointer.
-     * @param cbPtr Callback pointer.
-     * @param memPtr Memory pointer.
-     */
-    static native void continuousQueryListenerApply(long envPtr, long cbPtr, long memPtr);
-
-    /**
-     * Create filter in native platform.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     * @return Pointer to created filter.
-     */
-    static native long continuousQueryFilterCreate(long envPtr, long memPtr);
-
-    /**
-     * Invoke remote filter.
-     *
-     * @param envPtr Environment pointer.
-     * @param filterPtr Filter pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    static native int continuousQueryFilterApply(long envPtr, long filterPtr, long memPtr);
-
-    /**
-     * Release remote  filter.
-     *
-     * @param envPtr Environment pointer.
-     * @param filterPtr Filter pointer.
-     */
-    static native void continuousQueryFilterRelease(long envPtr, long filterPtr);
-
-    /**
-     * Notify native data streamer about topology update.
-     *
-     * @param envPtr Environment pointer.
-     * @param ptr Data streamer native pointer.
-     * @param topVer Topology version.
-     * @param topSize Topology size.
-     */
-    static native void dataStreamerTopologyUpdate(long envPtr, long ptr, long topVer, int topSize);
-
-    /**
-     * Invoke stream receiver.
-     *
-     * @param envPtr Environment pointer.
-     * @param ptr Receiver native pointer.
-     * @param cache Cache object.
-     * @param memPtr Stream pointer.
-     * @param keepPortable Portable flag.
-     */
-    static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, Object cache, long memPtr,
-        boolean keepPortable);
-
-    /**
-     * Notify future with byte result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureByteResult(long envPtr, long futPtr, int res);
-
-    /**
-     * Notify future with boolean result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureBoolResult(long envPtr, long futPtr, int res);
-
-    /**
-     * Notify future with short result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureShortResult(long envPtr, long futPtr, int res);
-
-    /**
-     * Notify future with byte result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureCharResult(long envPtr, long futPtr, int res);
-
-    /**
-     * Notify future with int result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureIntResult(long envPtr, long futPtr, int res);
-
-    /**
-     * Notify future with float result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureFloatResult(long envPtr, long futPtr, float res);
-
-    /**
-     * Notify future with long result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureLongResult(long envPtr, long futPtr, long res);
-
-    /**
-     * Notify future with double result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param res Result.
-     */
-    static native void futureDoubleResult(long envPtr, long futPtr, double res);
-
-    /**
-     * Notify future with object result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param memPtr Memory pointer.
-     */
-    static native void futureObjectResult(long envPtr, long futPtr, long memPtr);
-
-    /**
-     * Notify future with null result.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     */
-    static native void futureNullResult(long envPtr, long futPtr);
-
-    /**
-     * Notify future with error.
-     *
-     * @param envPtr Environment pointer.
-     * @param futPtr Future pointer.
-     * @param memPtr Pointer to memory with error information.
-     */
-    static native void futureError(long envPtr, long futPtr, long memPtr);
-
-    /**
-     * Creates message filter and returns a pointer.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    static native long messagingFilterCreate(long envPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    static native int messagingFilterApply(long envPtr, long objPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Pointer.
-     */
-    static native void messagingFilterDestroy(long envPtr, long objPtr);
-
-    /**
-     * Creates event filter and returns a pointer.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     * @return Pointer.
-     */
-    static native long eventFilterCreate(long envPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Pointer.
-     * @param memPtr Memory pointer.
-     * @return Result.
-     */
-    static native int eventFilterApply(long envPtr, long objPtr, long memPtr);
-
-    /**
-     * @param envPtr Environment pointer.
-     * @param objPtr Pointer.
-     */
-    static native void eventFilterDestroy(long envPtr, long objPtr);
-
-    /**
-     * Sends node info to native target.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Ptr to a stream with serialized node.
-     */
-    static native void nodeInfo(long envPtr, long memPtr);
-
-    /**
-     * Kernal start callback.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Memory pointer.
-     */
-    static native void onStart(long envPtr, long memPtr);
-
-    /*
-     * Kernal stop callback.
-     *
-     * @param envPtr Environment pointer.
-     */
-    static native void onStop(long envPtr);
-
-    /**
-     * Lifecycle event callback.
-     *
-     * @param envPtr Environment pointer.
-     * @param ptr Holder pointer.
-     * @param evt Event.
-     */
-    static native void lifecycleEvent(long envPtr, long ptr, int evt);
-
-    /**
-     * Re-allocate external memory chunk.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Cross-platform pointer.
-     * @param cap Capacity.
-     */
-    static native void memoryReallocate(long envPtr, long memPtr, int cap);
-
-    /**
-     * Initializes native service.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Stream pointer.
-     * @return Pointer to the native platform service.
-     */
-    static native long serviceInit(long envPtr, long memPtr);
-
-    /**
-     * Executes native service.
-     *
-     * @param envPtr Environment pointer.
-     * @param svcPtr Pointer to the service in the native platform.
-     * @param memPtr Stream pointer.
-     */
-    static native void serviceExecute(long envPtr, long svcPtr, long memPtr);
-
-    /**
-     * Cancels native service.
-     *
-     * @param envPtr Environment pointer.
-     * @param svcPtr Pointer to the service in the native platform.
-     * @param memPtr Stream pointer.
-     */
-    static native void serviceCancel(long envPtr, long svcPtr, long memPtr);
-
-    /**
-     /**
-     * Invokes service method.
-     *
-     * @param envPtr Environment pointer.
-     * @param svcPtr Pointer to the service in the native platform.
-     * @param outMemPtr Output memory pointer.
-     * @param inMemPtr Input memory pointer.
-     */
-    static native void serviceInvokeMethod(long envPtr, long svcPtr, long outMemPtr, long inMemPtr);
-
-    /**
-     * Invokes cluster node filter.
-     *
-     * @param envPtr Environment pointer.
-     * @param memPtr Stream pointer.
-     */
-    static native int clusterNodeFilterApply(long envPtr, long memPtr);
-
-    /**
-     * Private constructor.
-     */
-    private PlatformCallbackUtils() {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.java
deleted file mode 100644
index 9273e29..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStream.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.platform.memory;
-
-import org.apache.ignite.internal.portable.streams.*;
-
-/**
- * Interop output stream,
- */
-public interface PlatformInputStream extends PortableInputStream {
-    /**
-     * Synchronize input. Must be called before start reading data from a memory changed by another platform.
-     */
-    public void synchronize();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.java
deleted file mode 100644
index 9d8f94e..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemory.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.memory;
-
-/**
- * Interop memory chunk.
- */
-public interface PlatformMemory extends AutoCloseable {
-    /**
-     * Gets input stream.
-     *
-     * @return Input stream.
-     */
-    public PlatformInputStream input();
-
-    /**
-     * Gets output stream.
-     *
-     * @return Output stream.
-     */
-    public PlatformOutputStream output();
-
-    /**
-     * Gets pointer which can be passed between platforms.
-     *
-     * @return Pointer.
-     */
-    public long pointer();
-
-    /**
-     * Gets data pointer.
-     *
-     * @return Data pointer.
-     */
-    public long data();
-
-    /**
-     * Gets capacity.
-     *
-     * @return Capacity.
-     */
-    public int capacity();
-
-    /**
-     * Gets length.
-     *
-     * @return Length.
-     */
-    public int length();
-
-    /**
-     * Reallocate memory chunk.
-     *
-     * @param cap Minimum capacity.
-     */
-    public void reallocate(int cap);
-
-    /**
-     * Close memory releasing it.
-     */
-    @Override void close();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java
deleted file mode 100644
index c2233a8..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformMemoryManager.java
+++ /dev/null
@@ -1,46 +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.memory;
-
-/**
- * Interop memory manager interface.
- */
-public interface PlatformMemoryManager {
-    /**
-     * Allocates memory.
-     *
-     * @return Memory.
-     */
-    public PlatformMemory allocate();
-
-    /**
-     * Allocates memory having at least the given capacity.
-     *
-     * @param cap Minimum capacity.
-     * @return Memory.
-     */
-    public PlatformMemory allocate(int cap);
-
-    /**
-     * Gets memory from existing pointer.
-     *
-     * @param memPtr Cross-platform memory pointer.
-     * @return Memory.
-     */
-    public PlatformMemory get(long memPtr);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.java
deleted file mode 100644
index eb2490a..0000000
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStream.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.platform.memory;
-
-import org.apache.ignite.internal.portable.streams.*;
-
-/**
- * Interop output stream.
- */
-public interface PlatformOutputStream extends PortableOutputStream {
-    /**
-     * Synchronize output stream with underlying memory
-     */
-    public void synchronize();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5877b301/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 66e87e2..f82fb0f 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
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.platform.utils;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.portable.*;
 import org.apache.ignite.internal.processors.platform.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
@@ -577,6 +578,28 @@ public class PlatformUtils {
     }
 
     /**
+     * Get GridGain platform processor.
+     *
+     * @param grid Ignite instance.
+     * @return Platform processor.
+     */
+    public static PlatformProcessor platformProcessor(Ignite grid) {
+        GridKernalContext ctx = ((IgniteKernal) grid).context();
+
+        return ctx.platform();
+    }
+
+    /**
+     * Gets interop context for the grid.
+     *
+     * @param grid Grid
+     * @return Context.
+     */
+    public static PlatformContext platformContext(Ignite grid) {
+        return platformProcessor(grid).context();
+    }
+
+    /**
      * Private constructor.
      */
     private PlatformUtils() {


[29/50] [abbrv] ignite git commit: IGNITE-1310: Platforms: moved cluster group to Ignite.

Posted by ak...@apache.org.
IGNITE-1310: Platforms: moved cluster group to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 712b29c856614bd914e6bf35c5203aab624c93e7
Parents: 7c2c02b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 13:29:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 13:29:08 2015 +0300

----------------------------------------------------------------------
 .../platform/cluster/PlatformClusterGroup.java  | 330 +++++++++++++++++++
 1 file changed, 330 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/712b29c8/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
new file mode 100644
index 0000000..1f2a002
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -0,0 +1,330 @@
+/*
+ * 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.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+
+import java.util.*;
+
+/**
+ * Interop projection.
+ */
+@SuppressWarnings({"UnusedDeclaration"})
+public class PlatformClusterGroup extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_ALL_METADATA = 1;
+
+    /** */
+    private static final int OP_FOR_ATTRIBUTE = 2;
+
+    /** */
+    private static final int OP_FOR_CACHE = 3;
+
+    /** */
+    private static final int OP_FOR_CLIENT = 4;
+
+    /** */
+    private static final int OP_FOR_DATA = 5;
+
+    /** */
+    private static final int OP_FOR_HOST = 6;
+
+    /** */
+    private static final int OP_FOR_NODE_IDS = 7;
+
+    /** */
+    private static final int OP_METADATA = 8;
+
+    /** */
+    private static final int OP_METRICS = 9;
+
+    /** */
+    private static final int OP_METRICS_FILTERED = 10;
+
+    /** */
+    private static final int OP_NODE_METRICS = 11;
+
+    /** */
+    private static final int OP_NODES = 12;
+
+    /** */
+    private static final int OP_PING_NODE = 13;
+
+    /** */
+    private static final int OP_TOPOLOGY = 14;
+
+    /** Projection. */
+    private final ClusterGroupEx prj;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param prj Projection.
+     */
+    public PlatformClusterGroup(PlatformContext platformCtx, ClusterGroupEx prj) {
+        super(platformCtx);
+
+        this.prj = prj;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_METRICS:
+                platformCtx.writeClusterMetrics(writer, prj.metrics());
+
+                break;
+
+            case OP_ALL_METADATA:
+                platformCtx.writeAllMetadata(writer);
+
+                break;
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ConstantConditions", "deprecation"})
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object obj) throws IgniteCheckedException {
+        switch (type) {
+            case OP_METRICS_FILTERED: {
+                Collection<UUID> ids = PlatformUtils.readCollection(reader);
+
+                platformCtx.writeClusterMetrics(writer, prj.forNodeIds(ids).metrics());
+
+                break;
+            }
+
+            case OP_NODES: {
+                long oldTopVer = reader.readLong();
+
+                long curTopVer = platformCtx.kernalContext().discovery().topologyVersion();
+
+                if (curTopVer > oldTopVer) {
+                    writer.writeBoolean(true);
+
+                    writer.writeLong(curTopVer);
+
+                    // At this moment topology version might have advanced, and due to this race
+                    // we return outdated top ver to the callee. But this race is benign, the only
+                    // possible side effect is that the user will re-request nodes and we will return
+                    // the same set of nodes but with more recent topology version.
+                    Collection<ClusterNode> nodes = prj.nodes();
+
+                    platformCtx.writeNodes(writer, nodes);
+                }
+                else
+                    // No discovery events since last invocation.
+                    writer.writeBoolean(false);
+
+                break;
+            }
+
+            case OP_NODE_METRICS: {
+                UUID nodeId = reader.readUuid();
+
+                long lastUpdateTime = reader.readLong();
+
+                // Ask discovery because node might have been filtered out of current projection.
+                ClusterNode node = platformCtx.kernalContext().discovery().node(nodeId);
+
+                ClusterMetrics metrics = null;
+
+                if (node != null) {
+                    ClusterMetrics metrics0 = node.metrics();
+
+                    long triggerTime = lastUpdateTime + platformCtx.kernalContext().config().getMetricsUpdateFrequency();
+
+                    metrics = metrics0.getLastUpdateTime() > triggerTime ? metrics0 : null;
+                }
+
+                platformCtx.writeClusterMetrics(writer, metrics);
+
+                break;
+            }
+
+            case OP_METADATA: {
+                int typeId = reader.readInt();
+
+                platformCtx.writeMetadata(writer, typeId);
+
+                break;
+            }
+
+            case OP_TOPOLOGY: {
+                long topVer = reader.readLong();
+
+                platformCtx.writeNodes(writer, topology(topVer));
+
+                break;
+            }
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_PING_NODE:
+                return pingNode(reader.readUuid()) ? TRUE : FALSE;
+        }
+
+        return throwUnsupported(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object processInOpObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_FOR_NODE_IDS: {
+                Collection<UUID> ids = PlatformUtils.readCollection(reader);
+
+                return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forNodeIds(ids));
+            }
+
+            case OP_FOR_ATTRIBUTE:
+                return new PlatformClusterGroup(platformCtx,
+                    (ClusterGroupEx)prj.forAttribute(reader.readString(), reader.readString()));
+
+            case OP_FOR_CACHE: {
+                String cacheName = reader.readString();
+
+                return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forCacheNodes(cacheName));
+            }
+
+            case OP_FOR_CLIENT: {
+                String cacheName = reader.readString();
+
+                return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forClientNodes(cacheName));
+            }
+
+            case OP_FOR_DATA: {
+                String cacheName = reader.readString();
+
+                return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forDataNodes(cacheName));
+            }
+
+            case OP_FOR_HOST: {
+                UUID nodeId = reader.readUuid();
+
+                ClusterNode node = prj.node(nodeId);
+
+                return new PlatformClusterGroup(platformCtx, (ClusterGroupEx) prj.forHost(node));
+            }
+
+            default:
+                return throwUnsupported(type);
+        }
+    }
+
+    /**
+     * @param exclude Projection to exclude.
+     * @return New projection.
+     */
+    public PlatformClusterGroup forOthers(PlatformClusterGroup exclude) {
+        return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forOthers(exclude.prj));
+    }
+
+    /**
+     * @return New projection.
+     */
+    public PlatformClusterGroup forRemotes() {
+        return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRemotes());
+    }
+
+    /**
+     * @return New projection.
+     */
+    public PlatformClusterGroup forDaemons() {
+        return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forDaemons());
+    }
+
+    /**
+     * @return New projection.
+     */
+    public PlatformClusterGroup forRandom() {
+        return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRandom());
+    }
+
+    /**
+     * @return New projection.
+     */
+    public PlatformClusterGroup forOldest() {
+        return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forOldest());
+    }
+
+    /**
+     * @return New projection.
+     */
+    public PlatformClusterGroup forYoungest() {
+        return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forYoungest());
+    }
+
+    /**
+     * @return Projection.
+     */
+    public ClusterGroupEx projection() {
+        return prj;
+    }
+
+    /**
+     * Resets local I/O, job, and task execution metrics.
+     */
+    public void resetMetrics() {
+        assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group.
+
+        ((IgniteCluster)prj).resetMetrics();
+    }
+
+    /**
+     * Pings a remote node.
+     */
+    private boolean pingNode(UUID nodeId) {
+        assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group.
+
+        return ((IgniteCluster)prj).pingNode(nodeId);
+    }
+
+    /**
+     * Gets a topology by version. Returns {@code null} if topology history storage doesn't contain
+     * specified topology version (history currently keeps last {@code 1000} snapshots).
+     *
+     * @param topVer Topology version.
+     * @return Collection of grid nodes which represented by specified topology version,
+     * if it is present in history storage, {@code null} otherwise.
+     * @throws UnsupportedOperationException If underlying SPI implementation does not support
+     *      topology history. Currently only {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi}
+     *      supports topology history.
+     */
+    private Collection<ClusterNode> topology(long topVer) {
+        assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group.
+
+        return ((IgniteCluster)prj).topology(topVer);
+    }
+}


[21/50] [abbrv] ignite git commit: Moved platform cluster node filter to Ignite.

Posted by ak...@apache.org.
Moved platform cluster node filter to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 93b294260f8081424b86d28ffd4c0232df1d04f0
Parents: daa8796
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 12:25:52 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 12:25:52 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformBootstrap.java  |  1 -
 .../cluster/PlatformClusterNodeFilter.java      | 77 ++++++++++++++++++++
 2 files changed, 77 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/93b29426/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformBootstrap.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformBootstrap.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformBootstrap.java
index 319c670..ce26475 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformBootstrap.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/PlatformBootstrap.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.platform.*;
 
 /**
  * Platform bootstrap. Responsible for starting Ignite node with non-Java platform.

http://git-wip-us.apache.org/repos/asf/ignite/blob/93b29426/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
new file mode 100644
index 0000000..e12449f
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilter.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.cluster;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+
+/**
+ * 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);
+    }
+}


[38/50] [abbrv] ignite git commit: IGNITE-1314: Moved messaging to Ignite.

Posted by ak...@apache.org.
IGNITE-1314: Moved messaging to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 27a59cf8a18a9f12e42fd0dc54890f6e44d91515
Parents: f4c7107
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 09:54:10 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 09:54:10 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |  10 ++
 .../messaging/PlatformMessageFilter.java        | 109 +++++++++++++
 .../messaging/PlatformMessageLocalFilter.java   | 102 ++++++++++++
 .../platform/messaging/PlatformMessaging.java   | 162 +++++++++++++++++++
 4 files changed, 383 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/27a59cf8/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 461fb84..68e0e35 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
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.portable.*;
 import org.apache.ignite.internal.processors.cache.query.continuous.*;
 import org.apache.ignite.internal.processors.platform.cache.query.*;
@@ -154,4 +155,13 @@ public interface PlatformContext {
      * @return Filter.
      */
     public CacheContinuousQueryFilterEx createContinuousQueryFilter(Object filter);
+
+    /**
+     * Create remote message filter.
+     *
+     * @param filter Native filter.
+     * @param ptr Pointer of deployed native filter.
+     * @return Filter.
+     */
+    public GridLifecycleAwareMessageFilter<UUID, Object> createRemoteMessageFilter(Object filter, long ptr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a59cf8/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
new file mode 100644
index 0000000..8a433ac
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilter.java
@@ -0,0 +1,109 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+
+import java.util.*;
+
+/**
+ * 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;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a59cf8/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
new file mode 100644
index 0000000..71bb918
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
@@ -0,0 +1,102 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+
+import java.util.*;
+
+/**
+ * Interop local filter. Delegates apply to native platform, uses id to identify native target.
+ */
+public class PlatformMessageLocalFilter implements GridLifecycleAwareMessageFilter<UUID, Object> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    protected final long hnd;
+
+    /** */
+    protected final PlatformContext platformCtx;
+
+    /**
+     * Constructor.
+     *
+     * @param hnd Handle in the native platform.
+     * @param ctx Context.
+     */
+    public PlatformMessageLocalFilter(long hnd, PlatformContext ctx) {
+        assert ctx != null;
+        assert hnd != 0;
+
+        this.hnd = hnd;
+        this.platformCtx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(UUID uuid, Object m) {
+        try (PlatformMemory mem = platformCtx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = platformCtx.writer(out);
+
+            writer.writeObject(uuid);
+            writer.writeObject(m);
+
+            out.synchronize();
+
+            int res = platformCtx.gateway().messagingFilterApply(hnd, mem.pointer());
+
+            return res != 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        platformCtx.gateway().messagingFilterDestroy(hnd);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initialize(GridKernalContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        PlatformMessageLocalFilter filter = (PlatformMessageLocalFilter)o;
+
+        return hnd == filter.hnd;
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return (int)(hnd ^ (hnd >>> 32));
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a59cf8/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
new file mode 100644
index 0000000..ffc2ab3
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
@@ -0,0 +1,162 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.lang.*;
+
+import java.util.*;
+
+/**
+ * Interop messaging.
+ */
+public class PlatformMessaging extends PlatformAbstractTarget {
+    /** */
+    public static final int OP_LOC_LISTEN = 1;
+
+    /** */
+    public static final int OP_REMOTE_LISTEN = 2;
+
+    /** */
+    public static final int OP_SEND = 3;
+
+    /** */
+    public static final int OP_SEND_MULTI = 4;
+
+    /** */
+    public static final int OP_SEND_ORDERED = 5;
+
+    /** */
+    public static final int OP_STOP_LOC_LISTEN = 6;
+
+    /** */
+    public static final int OP_STOP_REMOTE_LISTEN = 7;
+
+    /** */
+    private final IgniteMessaging messaging;
+
+    /**
+     * Ctor.
+     *
+     * @param platformCtx Context.
+     * @param messaging Ignite messaging.
+     */
+    public PlatformMessaging(PlatformContext platformCtx, IgniteMessaging messaging) {
+        super(platformCtx);
+
+        assert messaging != null;
+
+        this.messaging = messaging;
+    }
+
+    /**
+     * Gets messaging with asynchronous mode enabled.
+     *
+     * @return Messaging with asynchronous mode enabled.
+     */
+    public PlatformMessaging withAsync() {
+        if (messaging.isAsync())
+            return this;
+
+        return new PlatformMessaging (platformCtx, messaging.withAsync());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_SEND:
+                messaging.send(reader.readObjectDetached(), reader.readObjectDetached());
+
+                return TRUE;
+
+            case OP_SEND_MULTI:
+                messaging.send(reader.readObjectDetached(), PlatformUtils.readCollection(reader));
+
+                return TRUE;
+
+            case OP_SEND_ORDERED:
+                messaging.sendOrdered(reader.readObjectDetached(), reader.readObjectDetached(), reader.readLong());
+
+                return TRUE;
+
+            case OP_LOC_LISTEN: {
+                PlatformMessageLocalFilter filter = new PlatformMessageLocalFilter(reader.readLong(), platformCtx);
+
+                Object topic = reader.readObjectDetached();
+
+                messaging.localListen(topic, filter);
+
+                return TRUE;
+            }
+
+            case OP_STOP_LOC_LISTEN: {
+                PlatformMessageLocalFilter filter = new PlatformMessageLocalFilter(reader.readLong(), platformCtx);
+
+                Object topic = reader.readObjectDetached();
+
+                messaging.stopLocalListen(topic, filter);
+
+                return TRUE;
+            }
+
+            case OP_STOP_REMOTE_LISTEN: {
+                messaging.stopRemoteListen(reader.readUuid());
+
+                return TRUE;
+            }
+
+            default:
+                throw new IgniteCheckedException("Unsupported operation type: " + type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_REMOTE_LISTEN:{
+                Object nativeFilter = reader.readObjectDetached();
+
+                long ptr = reader.readLong();  // interop pointer
+
+                Object topic = reader.readObjectDetached();
+
+                GridLifecycleAwareMessageFilter filter = platformCtx.createRemoteMessageFilter(nativeFilter, ptr);
+
+                UUID listenId = messaging.remoteListen(topic, filter);
+
+                writer.writeUuid(listenId);
+
+                break;
+            }
+
+            default:
+                throw new IgniteCheckedException("Unsupported operation type: " + type);
+        }
+    }
+
+    /** <inheritDoc /> */
+    @Override protected IgniteFuture currentFuture() throws IgniteCheckedException {
+        return messaging.future();
+    }
+}
\ No newline at end of file


[47/50] [abbrv] ignite git commit: IGNITE-1319: Moved platform services to Ignite.

Posted by ak...@apache.org.
IGNITE-1319: Moved platform services to Ignite.


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

Branch: refs/heads/ignite-843
Commit: e2f522ba0c450e5045393824cc66bceeba716628
Parents: 02f2465
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 16:23:15 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 16:23:15 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |   9 +
 .../platform/services/PlatformService.java      |  44 ++++
 .../platform/dotnet/PlatformDotNetService.java  |  27 ++
 .../dotnet/PlatformDotNetServiceImpl.java       |  47 ++++
 .../services/PlatformAbstractService.java       | 223 ++++++++++++++++
 .../platform/services/PlatformServices.java     | 252 +++++++++++++++++++
 6 files changed, 602 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e2f522ba/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 9b4a891..064cd91 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
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.platform.cache.query.*;
 import org.apache.ignite.internal.processors.platform.callback.*;
 import org.apache.ignite.internal.processors.platform.compute.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.stream.*;
 import org.jetbrains.annotations.*;
 
@@ -260,4 +261,12 @@ public interface PlatformContext {
      * @return Stream receiver.
      */
     public StreamReceiver createStreamReceiver(Object rcv, long ptr, boolean keepPortable);
+
+    /**
+     * Create cluster node filter.
+     *
+     * @param filter Native filter.
+     * @return Cluster node filter.
+     */
+    public IgnitePredicate<ClusterNode> createClusterNodeFilter(Object filter);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2f522ba/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformService.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformService.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformService.java
new file mode 100644
index 0000000..7b1daad
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformService.java
@@ -0,0 +1,44 @@
+/*
+ * 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.services;
+
+import org.apache.ignite.*;
+import org.apache.ignite.services.*;
+
+/**
+ * Base class for all platform services.
+ */
+public interface PlatformService extends Service {
+    /**
+     * Invokes native service method.
+     *
+     * @param mthdName Method name.
+     * @param srvKeepPortable Server keep portable flag.
+     * @param args Arguments.
+     * @return Resulting data.
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    public Object invokeMethod(String mthdName, boolean srvKeepPortable, Object[] args) throws IgniteCheckedException;
+
+    /**
+     * Gets native pointer.
+     *
+     * @return Native pointer.
+     */
+    public long pointer();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2f522ba/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetService.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetService.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetService.java
new file mode 100644
index 0000000..7c61cf8
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetService.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.dotnet;
+
+import org.apache.ignite.internal.processors.platform.services.*;
+
+/**
+ * Marker interface to denote a service implemented on .Net platform.
+ */
+public interface PlatformDotNetService extends PlatformService {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2f522ba/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetServiceImpl.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetServiceImpl.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetServiceImpl.java
new file mode 100644
index 0000000..74e143d
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetServiceImpl.java
@@ -0,0 +1,47 @@
+/*
+ * 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.dotnet;
+
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.services.*;
+
+/**
+ * Interop .Net service.
+ */
+public class PlatformDotNetServiceImpl extends PlatformAbstractService implements PlatformDotNetService {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Default constructor for serialization.
+     */
+    public PlatformDotNetServiceImpl() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param svc Service.
+     * @param ctx Context.
+     * @param srvKeepPortable Whether to keep objects portable on server if possible.
+     */
+    public PlatformDotNetServiceImpl(Object svc, PlatformContext ctx, boolean srvKeepPortable) {
+        super(svc, ctx, srvKeepPortable);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2f522ba/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
new file mode 100644
index 0000000..d53b9b5
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
@@ -0,0 +1,223 @@
+/*
+ * 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.services;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.services.*;
+
+import java.io.*;
+
+/**
+ * Base platform service implementation.
+ */
+public abstract class PlatformAbstractService implements PlatformService, Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** .Net portable service. */
+    protected Object svc;
+
+    /** Whether to keep objects portable on server if possible. */
+    protected boolean srvKeepPortable;
+
+    /** Pointer to deployed service. */
+    protected transient long ptr;
+
+    /** Context. */
+    protected transient PlatformContext platformCtx;
+
+    /**
+     * Default constructor for serialization.
+     */
+    public PlatformAbstractService() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param svc Service.
+     * @param ctx Context.
+     * @param srvKeepPortable Whether to keep objects portable on server if possible.
+     */
+    public PlatformAbstractService(Object svc, PlatformContext ctx, boolean srvKeepPortable) {
+        assert svc != null;
+        assert ctx != null;
+
+        this.svc = svc;
+        this.platformCtx = ctx;
+        this.srvKeepPortable = srvKeepPortable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        assert ptr == 0;
+        assert platformCtx != null;
+
+        try (PlatformMemory mem = platformCtx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = platformCtx.writer(out);
+
+            writer.writeBoolean(srvKeepPortable);
+            writer.writeObject(svc);
+
+            writeServiceContext(ctx, writer);
+
+            out.synchronize();
+
+            ptr = platformCtx.gateway().serviceInit(mem.pointer());
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        assert ptr != 0;
+        assert platformCtx != null;
+
+        try (PlatformMemory mem = platformCtx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = platformCtx.writer(out);
+
+            writer.writeBoolean(srvKeepPortable);
+
+            writeServiceContext(ctx, writer);
+
+            out.synchronize();
+
+            platformCtx.gateway().serviceExecute(ptr, mem.pointer());
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        assert ptr != 0;
+        assert platformCtx != null;
+
+        try (PlatformMemory mem = platformCtx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            PortableRawWriterEx writer = platformCtx.writer(out);
+
+            writer.writeBoolean(srvKeepPortable);
+
+            writeServiceContext(ctx, writer);
+
+            out.synchronize();
+
+            platformCtx.gateway().serviceCancel(ptr, mem.pointer());
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * Writes service context.
+     *
+     * @param ctx Context.
+     * @param writer Writer.
+     */
+    private void writeServiceContext(ServiceContext ctx, PortableRawWriterEx writer) {
+        writer.writeString(ctx.name());
+        writer.writeUuid(ctx.executionId());
+        writer.writeBoolean(ctx.isCancelled());
+        writer.writeString(ctx.cacheName());
+        writer.writeObject(ctx.affinityKey());
+    }
+
+    /** {@inheritDoc} */
+    @Override public long pointer() {
+        assert ptr != 0;
+
+        return ptr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object invokeMethod(String mthdName, boolean srvKeepPortable, Object[] args)
+        throws IgniteCheckedException {
+        assert ptr != 0;
+        assert platformCtx != null;
+
+        try (PlatformMemory outMem = platformCtx.memory().allocate()) {
+            PlatformOutputStream out = outMem.output();
+            PortableRawWriterEx writer = platformCtx.writer(out);
+
+            writer.writeBoolean(srvKeepPortable);
+            writer.writeString(mthdName);
+
+            if (args == null)
+                writer.writeBoolean(false);
+            else {
+                writer.writeBoolean(true);
+                writer.writeInt(args.length);
+
+                for (Object arg : args)
+                    writer.writeObjectDetached(arg);
+            }
+
+            out.synchronize();
+
+            try (PlatformMemory inMem = platformCtx.memory().allocate()) {
+                PlatformInputStream in = inMem.input();
+
+                PortableRawReaderEx reader = platformCtx.reader(in);
+
+                platformCtx.gateway().serviceInvokeMethod(ptr, outMem.pointer(), inMem.pointer());
+
+                in.synchronize();
+
+                return PlatformUtils.readInvocationResult(platformCtx, reader);
+            }
+        }
+    }
+
+    /**
+     * @param ignite Ignite instance.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    @IgniteInstanceResource
+    public void setIgniteInstance(Ignite ignite) {
+        platformCtx = PlatformUtils.platformContext(ignite);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        svc = in.readObject();
+        srvKeepPortable = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(svc);
+        out.writeBoolean(srvKeepPortable);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2f522ba/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
new file mode 100644
index 0000000..d0956f9
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -0,0 +1,252 @@
+/*
+ * 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.services;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.dotnet.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.services.*;
+
+import java.util.*;
+
+/**
+ * Interop services.
+ */
+@SuppressWarnings({"UnusedDeclaration"})
+public class PlatformServices extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_DOTNET_DEPLOY = 1;
+
+    /** */
+    private static final int OP_DOTNET_DEPLOY_MULTIPLE = 2;
+
+    /** */
+    private static final int OP_DOTNET_SERVICES = 3;
+
+    /** */
+    private static final int OP_DOTNET_INVOKE = 4;
+
+    /** */
+    private static final int OP_DESCRIPTORS = 5;
+
+    /** */
+    private final IgniteServices services;
+
+    /** Server keep portable flag. */
+    private final boolean srvKeepPortable;
+
+    /**
+     * Ctor.
+     *
+     * @param platformCtx Context.
+     * @param services Services facade.
+     * @param srvKeepPortable Server keep portable flag.
+     */
+    public PlatformServices(PlatformContext platformCtx, IgniteServices services, boolean srvKeepPortable) {
+        super(platformCtx);
+
+        assert services != null;
+
+        this.services = services;
+        this.srvKeepPortable = srvKeepPortable;
+    }
+
+    /**
+     * Gets services with asynchronous mode enabled.
+     *
+     * @return Services with asynchronous mode enabled.
+     */
+    public PlatformServices withAsync() {
+        if (services.isAsync())
+            return this;
+
+        return new PlatformServices(platformCtx, services.withAsync(), srvKeepPortable);
+    }
+
+    /**
+     * Gets services with server "keep portable" mode enabled.
+     *
+     * @return Services with server "keep portable" mode enabled.
+     */
+    public PlatformServices withServerKeepPortable() {
+        return srvKeepPortable ? this : new PlatformServices(platformCtx, services, true);
+    }
+
+    /**
+     * Cancels service deployment.
+     *
+     * @param name Name of service to cancel.
+     */
+    public void cancel(String name) {
+        services.cancel(name);
+    }
+
+    /**
+     * Cancels all deployed services.
+     */
+    public void cancelAll() {
+        services.cancelAll();
+    }
+
+    /**
+     * Gets a remote handle on the service.
+     *
+     * @param name Service name.
+     * @param sticky Whether or not Ignite should always contact the same remote service.
+     * @return Either proxy over remote service or local service if it is deployed locally.
+     */
+    public Object dotNetServiceProxy(String name, boolean sticky) {
+        return services.serviceProxy(name, PlatformDotNetService.class, sticky);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_DOTNET_DEPLOY: {
+                ServiceConfiguration cfg = new ServiceConfiguration();
+
+                cfg.setName(reader.readString());
+                cfg.setService(new PlatformDotNetServiceImpl(reader.readObjectDetached(), platformCtx, srvKeepPortable));
+                cfg.setTotalCount(reader.readInt());
+                cfg.setMaxPerNodeCount(reader.readInt());
+                cfg.setCacheName(reader.readString());
+                cfg.setAffinityKey(reader.readObjectDetached());
+
+                Object filter = reader.readObjectDetached();
+
+                if (filter != null)
+                    cfg.setNodeFilter(platformCtx.createClusterNodeFilter(filter));
+
+                services.deploy(cfg);
+
+                return TRUE;
+            }
+
+            case OP_DOTNET_DEPLOY_MULTIPLE: {
+                String name = reader.readString();
+                Object svc = reader.readObjectDetached();
+                int totalCnt = reader.readInt();
+                int maxPerNodeCnt = reader.readInt();
+
+                services.deployMultiple(name, new PlatformDotNetServiceImpl(svc, platformCtx, srvKeepPortable),
+                    totalCnt, maxPerNodeCnt);
+
+                return TRUE;
+            }
+        }
+
+        return super.processInOp(type, reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_DOTNET_SERVICES: {
+                Collection<Service> svcs = services.services(reader.readString());
+
+                PlatformUtils.writeNullableCollection(writer, svcs,
+                    new PlatformWriterClosure<Service>() {
+                        @Override public void write(PortableRawWriterEx writer, Service svc) {
+                            writer.writeLong(((PlatformService) svc).pointer());
+                        }
+                    },
+                    new IgnitePredicate<Service>() {
+                        @Override public boolean apply(Service svc) {
+                            return svc instanceof PlatformDotNetService;
+                        }
+                    }
+                );
+
+                return;
+            }
+
+            case OP_DOTNET_INVOKE: {
+                assert arg != null;
+                assert arg instanceof PlatformDotNetService;
+
+                String mthdName = reader.readString();
+
+                Object[] args;
+
+                if (reader.readBoolean()) {
+                    args = new Object[reader.readInt()];
+
+                    for (int i = 0; i < args.length; i++)
+                        args[i] = reader.readObjectDetached();
+                }
+                else
+                    args = null;
+
+                try {
+                    Object result = ((PlatformDotNetService)arg).invokeMethod(mthdName, srvKeepPortable, args);
+
+                    PlatformUtils.writeInvocationResult(writer, result, null);
+                }
+                catch (Exception e) {
+                    PlatformUtils.writeInvocationResult(writer, null, e);
+                }
+
+                return;
+            }
+        }
+
+        super.processInOutOp(type, reader, writer, arg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_DESCRIPTORS: {
+                Collection<ServiceDescriptor> descs = services.serviceDescriptors();
+
+                PlatformUtils.writeCollection(writer, descs, new PlatformWriterClosure<ServiceDescriptor>() {
+                    @Override public void write(PortableRawWriterEx writer, ServiceDescriptor d) {
+                        writer.writeString(d.name());
+                        writer.writeString(d.cacheName());
+                        writer.writeInt(d.maxPerNodeCount());
+                        writer.writeInt(d.totalCount());
+                        writer.writeUuid(d.originNodeId());
+                        writer.writeObject(d.affinityKey());
+
+                        Map<UUID, Integer> top = d.topologySnapshot();
+
+                        PlatformUtils.writeMap(writer, top, new PlatformWriterBiClosure<UUID, Integer>() {
+                            @Override public void write(PortableRawWriterEx writer, UUID key, Integer val) {
+                                writer.writeUuid(key);
+                                writer.writeInt(val);
+                            }
+                        });
+                    }
+                });
+
+                return;
+            }
+        }
+
+        super.processOutOp(type, writer);
+    }
+
+    /** <inheritDoc /> */
+    @Override protected IgniteFuture currentFuture() throws IgniteCheckedException {
+        return services.future();
+    }
+}


[06/50] [abbrv] ignite git commit: Platforms: moved several cache store classes to Ignite.

Posted by ak...@apache.org.
Platforms: moved several cache store classes to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 2938a9b4a06b0e0a9f19625ce1b168e4a4a2f87d
Parents: 536af49
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 17:11:46 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 17:11:46 2015 +0300

----------------------------------------------------------------------
 .../cache/store/PlatformCacheStore.java         | 25 ++++++++
 .../cache/store/PlatformCacheStoreCallback.java | 61 ++++++++++++++++++++
 2 files changed, 86 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2938a9b4/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java
new file mode 100644
index 0000000..6e0c1d9
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStore.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.cache.store;
+
+/**
+ * Marker interface denoting that this instance is platform cache store.
+ */
+public interface PlatformCacheStore {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2938a9b4/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
new file mode 100644
index 0000000..32e0ab7
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
@@ -0,0 +1,61 @@
+/*
+ * 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.store;
+
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+
+/**
+ * Platform cache store callback.
+ */
+public abstract class PlatformCacheStoreCallback {
+    /** Context. */
+    protected final PlatformContext ctx;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Context.
+     */
+    protected PlatformCacheStoreCallback(PlatformContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /**
+     * Invoke the callback.
+     *
+     * @param memPtr Memory pointer.
+     */
+    public void invoke(long memPtr) {
+        if (memPtr > 0) {
+            try (PlatformMemory mem = ctx.memory().get(memPtr)) {
+                PortableRawReaderEx reader = ctx.reader(mem);
+
+                invoke0(reader);
+            }
+        }
+    }
+
+    /**
+     * Internal invoke routine.
+     *
+     * @param reader Reader.
+     */
+    protected abstract void invoke0(PortableRawReaderEx reader);
+}


[12/50] [abbrv] ignite git commit: Moved two platform utility methods to Ignite.

Posted by ak...@apache.org.
Moved two platform utility methods to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 57184803fc64049e18d27d04777687e3f1f7e42d
Parents: 5877b30
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 10:07:55 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 10:07:55 2015 +0300

----------------------------------------------------------------------
 .../platform/utils/PlatformUtils.java           | 55 ++++++++++++++++++++
 1 file changed, 55 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/57184803/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 f82fb0f..0777f9a 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
@@ -600,6 +600,61 @@ public class PlatformUtils {
     }
 
     /**
+     * Reallocate arbitrary memory chunk.
+     *
+     * @param memPtr Memory pointer.
+     * @param cap Capacity.
+     */
+    public static void reallocate(long memPtr, int cap) {
+        PlatformMemoryUtils.reallocate(memPtr, cap);
+    }
+
+    /**
+     * Get error data.
+     *
+     * @param err Error.
+     * @return Error data.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public static byte[] errorData(Throwable err) {
+        if (err instanceof PlatformExtendedException) {
+            PlatformContext ctx = ((PlatformExtendedException)err).context();
+
+            try (PlatformMemory mem = ctx.memory().allocate()) {
+                // Write error data.
+                PlatformOutputStream out = mem.output();
+
+                PortableRawWriterEx writer = ctx.writer(out);
+
+                try {
+                    PlatformUtils.writeErrorData(err, writer, ctx.kernalContext().log(PlatformContext.class));
+                }
+                finally {
+                    out.synchronize();
+                }
+
+                // Read error data into separate array.
+                PlatformInputStream in = mem.input();
+
+                in.synchronize();
+
+                int len = in.remaining();
+
+                assert len > 0;
+
+                byte[] arr = in.array();
+                byte[] res = new byte[len];
+
+                System.arraycopy(arr, 0, res, 0, len);
+
+                return res;
+            }
+        }
+        else
+            return null;
+    }
+
+    /**
      * Private constructor.
      */
     private PlatformUtils() {


[39/50] [abbrv] ignite git commit: IGNITE-1315: Moved events to Ignite.

Posted by ak...@apache.org.
IGNITE-1315: Moved events to Ignite.


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

Branch: refs/heads/ignite-843
Commit: a0eeea6fb61c203f5a3ec7b7e394839223c27eb3
Parents: 27a59cf
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 11:00:44 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 11:00:44 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformAwareEventFilter.java      |   4 +-
 .../processors/platform/PlatformContext.java    |  34 ++
 .../platform/events/PlatformEventFilter.java    | 161 ++++++++
 .../platform/events/PlatformEvents.java         | 388 +++++++++++++++++++
 4 files changed, 586 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a0eeea6f/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
index f056bbf..b09d889 100644
--- 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
@@ -21,10 +21,12 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 
+import java.util.*;
+
 /**
  * Special version of predicate for events with initialize/close callbacks.
  */
-public interface PlatformAwareEventFilter<E extends Event> extends IgnitePredicate<E> {
+public interface PlatformAwareEventFilter<E extends Event> extends IgnitePredicate<E>, IgniteBiPredicate<UUID, E> {
     /**
      * Initializes the filter.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a0eeea6f/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 68e0e35..82a42d5 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,6 +18,7 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.portable.*;
@@ -164,4 +165,37 @@ public interface PlatformContext {
      * @return Filter.
      */
     public GridLifecycleAwareMessageFilter<UUID, Object> createRemoteMessageFilter(Object filter, long ptr);
+
+    /**
+     * Check whether the given event type is supported.
+     *
+     * @param evtTyp Event type.
+     * @return {@code True} if supported.
+     */
+    public boolean isEventTypeSupported(int evtTyp);
+
+    /**
+     * Write event.
+     *
+     * @param writer Writer.
+     * @param event Event.
+     */
+    public void writeEvent(PortableRawWriterEx writer, EventAdapter event);
+
+    /**
+     * Create local event filter.
+     *
+     * @param hnd Native handle.
+     * @return Filter.
+     */
+    public <E extends Event> PlatformAwareEventFilter<E> createLocalEventFilter(long hnd);
+
+    /**
+     * Create remote event filter.
+     *
+     * @param pred Native predicate.
+     * @param types Event types.
+     * @return Filter.
+     */
+    public <E extends Event> PlatformAwareEventFilter<E> createRemoteEventFilter(Object pred, final int... types);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a0eeea6f/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
new file mode 100644
index 0000000..7255dbb
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilter.java
@@ -0,0 +1,161 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+
+import java.util.*;
+
+/**
+ * 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/a0eeea6f/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
new file mode 100644
index 0000000..befc3bd
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
@@ -0,0 +1,388 @@
+/*
+ * 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.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop events.
+ */
+public class PlatformEvents extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_REMOTE_QUERY = 1;
+
+    /** */
+    private static final int OP_REMOTE_LISTEN = 2;
+
+    /** */
+    private static final int OP_STOP_REMOTE_LISTEN = 3;
+
+    /** */
+    private static final int OP_WAIT_FOR_LOCAL = 4;
+
+    /** */
+    private static final int OP_LOCAL_QUERY = 5;
+
+    /** */
+    private static final int OP_RECORD_LOCAL = 6;
+
+    /** */
+    private static final int OP_ENABLE_LOCAL = 8;
+
+    /** */
+    private static final int OP_DISABLE_LOCAL = 9;
+
+    /** */
+    private static final int OP_GET_ENABLED_EVENTS = 10;
+
+    /** */
+    private final IgniteEvents events;
+
+    /** */
+    private final EventResultWriter eventResWriter;
+
+    /** */
+    private final EventCollectionResultWriter eventColResWriter;
+
+    /**
+     * Ctor.
+     *
+     * @param platformCtx Context.
+     * @param events Ignite events.
+     */
+    public PlatformEvents(PlatformContext platformCtx, IgniteEvents events) {
+        super(platformCtx);
+
+        assert events != null;
+
+        this.events = events;
+
+        eventResWriter = new EventResultWriter(platformCtx);
+        eventColResWriter = new EventCollectionResultWriter(platformCtx);
+    }
+
+    /**
+     * Gets events with asynchronous mode enabled.
+     *
+     * @return Events with asynchronous mode enabled.
+     */
+    public PlatformEvents withAsync() {
+        if (events.isAsync())
+            return this;
+
+        return new PlatformEvents(platformCtx, events.withAsync());
+    }
+
+    /**
+     * Adds an event listener for local events.
+     *
+     * @param hnd Interop listener handle.
+     * @param type Event type.
+     */
+    @SuppressWarnings({"unchecked"})
+    public void localListen(long hnd, int type) {
+        events.localListen(localFilter(hnd), type);
+    }
+
+    /**
+     * Removes an event listener for local events.
+     *
+     * @param hnd Interop listener handle.
+     */
+    @SuppressWarnings({"UnusedDeclaration", "unchecked"})
+    public boolean stopLocalListen(long hnd) {
+        return events.stopLocalListen(localFilter(hnd));
+    }
+
+    /**
+     * Check if event is enabled.
+     *
+     * @param type Event type.
+     * @return {@code True} if event of passed in type is enabled.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public boolean isEnabled(int type) {
+        return events.isEnabled(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_RECORD_LOCAL:
+                // TODO: GG-10244
+                break;
+
+            case OP_ENABLE_LOCAL:
+
+                events.enableLocal(readEventTypes(reader));
+
+                return TRUE;
+
+            case OP_DISABLE_LOCAL:
+
+                events.disableLocal(readEventTypes(reader));
+
+                return TRUE;
+
+            case OP_STOP_REMOTE_LISTEN:
+                events.stopRemoteListen(reader.readUuid());
+
+                return TRUE;
+        }
+
+        throw new IgniteCheckedException("Unsupported operation type: " + type);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_LOCAL_QUERY: {
+                Collection<EventAdapter> result =
+                    events.localQuery(F.<EventAdapter>alwaysTrue(), readEventTypes(reader));
+
+                writer.writeInt(result.size());
+
+                for (EventAdapter e : result)
+                    platformCtx.writeEvent(writer, e);
+
+                break;
+            }
+
+            case OP_WAIT_FOR_LOCAL: {
+                boolean hasFilter = reader.readBoolean();
+
+                IgnitePredicate pred = hasFilter ? localFilter(reader.readLong()) : null;
+
+                int[] eventTypes = readEventTypes(reader);
+
+                EventAdapter result = (EventAdapter) events.waitForLocal(pred, eventTypes);
+
+                platformCtx.writeEvent(writer, result);
+
+                break;
+            }
+
+            case OP_REMOTE_LISTEN: {
+                int bufSize = reader.readInt();
+
+                long interval = reader.readLong();
+
+                boolean autoUnsubscribe = reader.readBoolean();
+
+                boolean hasLocFilter = reader.readBoolean();
+
+                PlatformAwareEventFilter locFilter = hasLocFilter ? localFilter(reader.readLong()) : null;
+
+                boolean hasRmtFilter = reader.readBoolean();
+
+                UUID listenId;
+
+                if (hasRmtFilter) {
+                    PlatformAwareEventFilter rmtFilter = platformCtx.createRemoteEventFilter(
+                        reader.readObjectDetached(), readEventTypes(reader));
+
+                    listenId = events.remoteListen(bufSize, interval, autoUnsubscribe, locFilter, rmtFilter);
+                }
+                else
+                    listenId = events.remoteListen(bufSize, interval, autoUnsubscribe, locFilter, null,
+                        readEventTypes(reader));
+
+                writer.writeUuid(listenId);
+
+                break;
+            }
+
+            case OP_REMOTE_QUERY: {
+                Object pred = reader.readObjectDetached();
+
+                long timeout = reader.readLong();
+
+                int[] types = readEventTypes(reader);
+
+                PlatformAwareEventFilter filter = platformCtx.createRemoteEventFilter(pred, types);
+
+                Collection<EventAdapter> result = events.remoteQuery(filter, timeout);
+
+                if (result == null)
+                    writer.writeInt(-1);
+                else {
+                    writer.writeInt(result.size());
+
+                    for (EventAdapter e : result)
+                        platformCtx.writeEvent(writer, e);
+                }
+
+                break;
+            }
+
+            default:
+                throw new IgniteCheckedException("Unsupported operation type: " + type);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_ENABLED_EVENTS:
+                writeEventTypes(events.enabledEvents(), writer);
+
+                break;
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /** <inheritDoc /> */
+    @Override protected IgniteFuture currentFuture() throws IgniteCheckedException {
+        return events.future();
+    }
+
+    /** <inheritDoc /> */
+    @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) {
+        switch (opId) {
+            case OP_WAIT_FOR_LOCAL:
+                return eventResWriter;
+
+            case OP_REMOTE_QUERY:
+                return eventColResWriter;
+        }
+
+        return null;
+    }
+
+    /**
+     *  Reads event types array.
+     *
+     * @param reader Reader
+     * @return Event types, or null.
+     */
+    private int[] readEventTypes(PortableRawReaderEx reader) {
+        return reader.readIntArray();
+    }
+
+    /**
+     *  Reads event types array.
+     *
+     * @param writer Writer
+     * @param types Types.
+     */
+    private void writeEventTypes(int[] types, PortableRawWriterEx writer) {
+        if (types == null) {
+            writer.writeIntArray(null);
+
+            return;
+        }
+
+        int[] resultTypes = new int[types.length];
+
+        int idx = 0;
+
+        for (int t : types)
+            if (platformCtx.isEventTypeSupported(t))
+                resultTypes[idx++] = t;
+
+        writer.writeIntArray(Arrays.copyOf(resultTypes, idx));
+    }
+
+    /**
+     * Creates an interop filter from handle.
+     *
+     * @param hnd Handle.
+     * @return Interop filter.
+     */
+    private PlatformAwareEventFilter localFilter(long hnd) {
+        return platformCtx.createLocalEventFilter(hnd);
+    }
+
+    /**
+     * Writes an EventBase.
+     */
+    private static class EventResultWriter implements PlatformFutureUtils.Writer {
+        /** */
+        private final PlatformContext platformCtx;
+
+        /**
+         * Constructor.
+         *
+         * @param platformCtx Context.
+         */
+        public EventResultWriter(PlatformContext platformCtx) {
+            assert platformCtx != null;
+
+            this.platformCtx = platformCtx;
+        }
+
+        /** <inheritDoc /> */
+        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+            platformCtx.writeEvent(writer, (EventAdapter)obj);
+        }
+
+        /** <inheritDoc /> */
+        @Override public boolean canWrite(Object obj, Throwable err) {
+            return obj instanceof EventAdapter && err == null;
+        }
+    }
+
+    /**
+     * Writes a collection of EventAdapter.
+     */
+    private static class EventCollectionResultWriter implements PlatformFutureUtils.Writer {
+        /** */
+        private final PlatformContext platformCtx;
+
+        /**
+         * Constructor.
+         *
+         * @param platformCtx Context.
+         */
+        public EventCollectionResultWriter(PlatformContext platformCtx) {
+            assert platformCtx != null;
+
+            this.platformCtx = platformCtx;
+        }
+
+        /** <inheritDoc /> */
+        @SuppressWarnings("unchecked")
+        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+            Collection<EventAdapter> events = (Collection<EventAdapter>)obj;
+
+            writer.writeInt(events.size());
+
+            for (EventAdapter e : events)
+                platformCtx.writeEvent(writer, e);
+        }
+
+        /** <inheritDoc /> */
+        @Override public boolean canWrite(Object obj, Throwable err) {
+            return obj instanceof Collection && err == null;
+        }
+    }
+}
+


[35/50] [abbrv] 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/3e30c863
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3e30c863
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3e30c863

Branch: refs/heads/ignite-843
Commit: 3e30c863bced6c77a8cffa022f966a50ceac19e1
Parents: 73ab5e2 c279fca
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 15:17:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 15:17:52 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/portable/PortableContext.java  | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[17/50] [abbrv] ignite git commit: IGNITE-1299: Implemented IGFS file unlock with retries.

Posted by ak...@apache.org.
IGNITE-1299: Implemented IGFS file unlock with retries.


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

Branch: refs/heads/ignite-843
Commit: 9fe3e8fd884f2f19bfe3fb39c9cda89c7ae495d8
Parents: e4ba2eb
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Aug 27 12:11:13 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 12:11:13 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteInternalFuture.java   | 10 ++++
 .../query/GridCacheQueryFutureAdapter.java      |  7 +++
 .../processors/igfs/IgfsMetaManager.java        | 43 +++++++--------
 .../internal/processors/igfs/IgfsUtils.java     | 52 ++++++++++++++++++
 .../util/future/GridFinishedFuture.java         |  5 ++
 .../internal/util/future/GridFutureAdapter.java | 58 +++++++++++++-------
 6 files changed, 131 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe3e8fd/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
index 2b7b821..74cfb06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
@@ -69,6 +69,16 @@ public interface IgniteInternalFuture<R> {
     public R get(long timeout, TimeUnit unit) throws IgniteCheckedException;
 
     /**
+     * Synchronously waits for completion of the computation and returns computation result ignoring interrupts.
+     *
+     * @return Computation result.
+     * @throws IgniteFutureCancelledCheckedException Subclass of {@link IgniteCheckedException} throws if computation
+     *     was cancelled.
+     * @throws IgniteCheckedException If computation failed.
+     */
+    public R getUninterruptibly() throws IgniteCheckedException;
+
+    /**
      * Cancels this future.
      *
      * @return {@code True} if future was canceled (i.e. was not finished prior to this call).

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe3e8fd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index 53017c9..ed5ad77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -479,6 +479,13 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
         return super.get(timeout, unit);
     }
 
+    /** {@inheritDoc} */
+    @Override public Collection<R> getUninterruptibly() throws IgniteCheckedException {
+        if (!isDone())
+            loadAllPages();
+
+        return super.getUninterruptibly();
+    }
 
     /**
      * @param nodeId Sender node id.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe3e8fd/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index b98c5d8..aabe503 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -478,49 +478,46 @@ public class IgfsMetaManager extends IgfsManager {
      * @param modificationTime Modification time to write to file info.
      * @throws IgniteCheckedException If failed.
      */
-    public void unlock(IgfsFileInfo info, long modificationTime) throws IgniteCheckedException {
+    public void unlock(final IgfsFileInfo info, final long modificationTime) throws IgniteCheckedException {
         assert validTxState(false);
         assert info != null;
 
         if (busyLock.enterBusy()) {
             try {
-                IgniteUuid lockId = info.lockId();
+                final IgniteUuid lockId = info.lockId();
 
                 if (lockId == null)
                     return;
 
                 // Temporary clear interrupted state for unlocking.
-                boolean interrupted = Thread.interrupted();
-
-                IgniteUuid fileId = info.id();
-
-                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                final boolean interrupted = Thread.interrupted();
 
                 try {
-                    // Lock file ID for this transaction.
-                    IgfsFileInfo oldInfo = info(fileId);
+                    IgfsUtils.doInTransactionWithRetries(metaCache, new IgniteOutClosureX<Void>() {
+                        @Override public Void applyx() throws IgniteCheckedException {
+                            IgniteUuid fileId = info.id();
 
-                    if (oldInfo == null)
-                        throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not found): " + fileId));
+                            // Lock file ID for this transaction.
+                            IgfsFileInfo oldInfo = info(fileId);
 
-                    if (!info.lockId().equals(oldInfo.lockId()))
-                        throw new IgniteCheckedException("Failed to unlock file (inconsistent file lock ID) [fileId=" + fileId +
-                            ", lockId=" + info.lockId() + ", actualLockId=" + oldInfo.lockId() + ']');
+                            if (oldInfo == null)
+                                throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not found): " + fileId));
 
-                    IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, null, modificationTime);
+                            if (!info.lockId().equals(oldInfo.lockId()))
+                                throw new IgniteCheckedException("Failed to unlock file (inconsistent file lock ID) [fileId=" + fileId +
+                                    ", lockId=" + info.lockId() + ", actualLockId=" + oldInfo.lockId() + ']');
 
-                    boolean put = metaCache.put(fileId, newInfo);
+                            IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, null, modificationTime);
 
-                    assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo + ']';
+                            boolean put = metaCache.put(fileId, newInfo);
 
-                    tx.commit();
-                }
-                catch (GridClosureException e) {
-                    throw U.cast(e);
+                            assert put : "Value was not stored in cache [fileId=" + fileId + ", newInfo=" + newInfo + ']';
+
+                            return null;
+                        }
+                    });
                 }
                 finally {
-                    tx.close();
-
                     assert validTxState(false);
 
                     if (interrupted)

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe3e8fd/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 8026a44..7449f31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -18,18 +18,31 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
 import java.lang.reflect.*;
 
+import static org.apache.ignite.IgniteSystemProperties.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
 /**
  * Common IGFS utility methods.
  */
 public class IgfsUtils {
+    /** Maximum number of file unlock transaction retries when topology changes. */
+    private static final int MAX_CACHE_TX_RETRIES = IgniteSystemProperties.getInteger(IGNITE_CACHE_RETRIES_COUNT, 100);
+
     /**
      * Converts any passed exception to IGFS exception.
      *
@@ -104,4 +117,43 @@ public class IgfsUtils {
 
         return user;
     }
+
+    /**
+     * Performs an operation with transaction with retries.
+     *
+     * @param cache Cache to do the transaction on.
+     * @param clo Closure.
+     * @return Result of closure execution.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public static <T> T doInTransactionWithRetries(IgniteInternalCache cache, IgniteOutClosureX<T> clo)
+        throws IgniteCheckedException {
+        assert cache != null;
+
+        int attempts = 0;
+
+        while (attempts < MAX_CACHE_TX_RETRIES) {
+            try (Transaction tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                T res = clo.applyx();
+
+                tx.commit();
+
+                return res;
+            }
+            catch (IgniteException | IgniteCheckedException e) {
+                ClusterTopologyException cte = X.cause(e, ClusterTopologyException.class);
+
+                if (cte != null)
+                    ((IgniteFutureImpl)cte.retryReadyFuture()).internalFuture().getUninterruptibly();
+                else
+                    throw U.cast(e);
+            }
+
+            attempts++;
+        }
+
+        throw new IgniteCheckedException("Failed to perform operation since max number of attempts " +
+            "exceeded. [maxAttempts=" + MAX_CACHE_TX_RETRIES + ']');
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe3e8fd/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
index 242e626..2adee90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
@@ -126,6 +126,11 @@ public class GridFinishedFuture<T> implements IgniteInternalFuture<T> {
     }
 
     /** {@inheritDoc} */
+    @Override public T getUninterruptibly() throws IgniteCheckedException {
+        return get();
+    }
+
+    /** {@inheritDoc} */
     @Override public void listen(IgniteInClosure<? super IgniteInternalFuture<T>> lsnr) {
         assert lsnr != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9fe3e8fd/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index f8caf22..91ce549 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -107,6 +107,43 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
 
     /** {@inheritDoc} */
     @Override public R get() throws IgniteCheckedException {
+        return get0(ignoreInterrupts);
+    }
+
+    /** {@inheritDoc} */
+    @Override public R getUninterruptibly() throws IgniteCheckedException {
+        return get0(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public R get(long timeout) throws IgniteCheckedException {
+        // Do not replace with static import, as it may not compile.
+        return get(timeout, TimeUnit.MILLISECONDS);
+    }
+
+    /** {@inheritDoc} */
+    @Override public R get(long timeout, TimeUnit unit) throws IgniteCheckedException {
+        A.ensure(timeout >= 0, "timeout cannot be negative: " + timeout);
+        A.notNull(unit, "unit");
+
+        try {
+            return get0(unit.toNanos(timeout));
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException("Got interrupted while waiting for future to complete.", e);
+        }
+    }
+
+    /**
+     * Internal get routine.
+     *
+     * @param ignoreInterrupts Whether to ignore interrupts.
+     * @return Result.
+     * @throws IgniteCheckedException If failed.
+     */
+    private R get0(boolean ignoreInterrupts) throws IgniteCheckedException {
         try {
             if (endTime == 0) {
                 if (ignoreInterrupts)
@@ -132,27 +169,6 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public R get(long timeout) throws IgniteCheckedException {
-        // Do not replace with static import, as it may not compile.
-        return get(timeout, TimeUnit.MILLISECONDS);
-    }
-
-    /** {@inheritDoc} */
-    @Override public R get(long timeout, TimeUnit unit) throws IgniteCheckedException {
-        A.ensure(timeout >= 0, "timeout cannot be negative: " + timeout);
-        A.notNull(unit, "unit");
-
-        try {
-            return get0(unit.toNanos(timeout));
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException("Got interrupted while waiting for future to complete.", e);
-        }
-    }
-
     /**
      * @param nanosTimeout Timeout (nanoseconds).
      * @return Result.


[02/50] [abbrv] 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/824cfa4b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/824cfa4b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/824cfa4b

Branch: refs/heads/ignite-843
Commit: 824cfa4b59d538eee5b3d53fb4af4c41e8412038
Parents: 8529e10 c9a5340
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 15:44:09 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 15:44:09 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheManyAsyncOperationsTest.java       | 2 +-
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java    | 1 +
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[07/50] [abbrv] ignite git commit: Restored backward compatibility for cache objects

Posted by ak...@apache.org.
Restored backward compatibility for cache objects


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

Branch: refs/heads/ignite-843
Commit: 38b3ffdaf2944cb908c45e22e3a1b1b834cf3b6c
Parents: 2938a9b
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Aug 26 15:39:12 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Aug 26 15:39:12 2015 -0700

----------------------------------------------------------------------
 .../processors/cacheobject/IgniteCacheObjectProcessor.java         | 2 +-
 .../processors/cacheobject/IgniteCacheObjectProcessorImpl.java     | 2 +-
 .../ignite/internal/processors/query/GridQueryProcessor.java       | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/38b3ffda/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index dc0d1e5..b9b6132 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -78,7 +78,7 @@ public interface IgniteCacheObjectProcessor extends GridProcessor {
      *
      * @return {@code true} If portable objects are enabled.
      */
-    public boolean isPortableEnabled();
+    public boolean isPortableEnabled(CacheConfiguration<?, ?> ccfg);
 
     /**
      * @param obj Portable object to get field from.

http://git-wip-us.apache.org/repos/asf/ignite/blob/38b3ffda/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 3e59b10..e8f7781 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -233,7 +233,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isPortableEnabled() {
+    @Override public boolean isPortableEnabled(CacheConfiguration<?, ?> ccfg) {
         return false;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/38b3ffda/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index f3ad4b2..c8b8166 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -140,7 +140,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     TypeId typeId;
 
-                    if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
+                    if (valCls == null || ctx.cacheObjects().isPortableEnabled(ccfg)) {
                         processPortableMeta(meta, desc);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));


[25/50] [abbrv] ignite git commit: Platforms: refactored metadata management methods.

Posted by ak...@apache.org.
Platforms: refactored metadata management methods.


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

Branch: refs/heads/ignite-843
Commit: a312934cb82a71145ed6018230a21302f871a1fa
Parents: 58f9fe4
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 13:17:20 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 13:17:20 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java         | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a312934c/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 90ed85d..504f79e 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
@@ -114,6 +114,21 @@ public interface PlatformContext {
     public void processMetadata(PortableRawReaderEx reader);
 
     /**
+     * Write metadata for the given type ID.
+     *
+     * @param writer Writer.
+     * @param typeId Type ID.
+     */
+    public void writeMetadata(PortableRawWriterEx writer, int typeId);
+
+    /**
+     * Write all available metadata.
+     *
+     * @param writer Writer.
+     */
+    public void writeAllMetadata(PortableRawWriterEx writer);
+
+    /**
      * Write cluster metrics.
      *
      * @param writer Writer.


[41/50] [abbrv] ignite git commit: IGNITE-1316: Moved compute to Ignite.

Posted by ak...@apache.org.
IGNITE-1316: Moved compute to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 5bbb8a334a9e6c1b4ad96457cd856036717702b4
Parents: a0eeea6
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 28 12:59:52 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 28 12:59:52 2015 +0300

----------------------------------------------------------------------
 .../processors/platform/PlatformContext.java    |  31 ++
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../platform/compute/PlatformJob.java           |  39 +++
 .../platform/compute/PlatformAbstractJob.java   | 154 +++++++++
 .../platform/compute/PlatformAbstractTask.java  | 202 ++++++++++++
 .../PlatformBalancingMultiClosureTask.java      |  78 +++++
 ...tformBalancingSingleClosureAffinityTask.java |  86 +++++
 .../PlatformBalancingSingleClosureTask.java     |  77 +++++
 .../PlatformBroadcastingMultiClosureTask.java   |  83 +++++
 .../PlatformBroadcastingSingleClosureTask.java  |  81 +++++
 .../platform/compute/PlatformClosureJob.java    | 102 ++++++
 .../platform/compute/PlatformCompute.java       | 323 +++++++++++++++++++
 .../platform/compute/PlatformFullJob.java       | 217 +++++++++++++
 .../platform/compute/PlatformFullTask.java      | 185 +++++++++++
 .../compute/PlatformNativeException.java        |  75 +++++
 .../platform/utils/PlatformUtils.java           |  75 +++++
 17 files changed, 1820 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/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 82a42d5..5275e0d 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,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
@@ -25,6 +26,7 @@ import org.apache.ignite.internal.portable.*;
 import org.apache.ignite.internal.processors.cache.query.continuous.*;
 import org.apache.ignite.internal.processors.platform.cache.query.*;
 import org.apache.ignite.internal.processors.platform.callback.*;
+import org.apache.ignite.internal.processors.platform.compute.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
 import org.jetbrains.annotations.*;
 
@@ -198,4 +200,33 @@ public interface PlatformContext {
      * @return Filter.
      */
     public <E extends Event> PlatformAwareEventFilter<E> createRemoteEventFilter(Object pred, final int... types);
+
+    /**
+     * Create native exception.
+     *
+     * @param cause Native cause.
+     * @return Exception.
+     */
+    // TODO: Some common interface must be used here.
+    public IgniteCheckedException createNativeException(Object cause);
+
+    /**
+     * Create job.
+     *
+     * @param task Task.
+     * @param ptr Pointer.
+     * @param job Native job.
+     * @return job.
+     */
+    public PlatformJob createJob(Object task, long ptr, @Nullable Object job);
+
+    /**
+     * Create closure job.
+     *
+     * @param task Task.
+     * @param ptr Pointer.
+     * @param job Native job.
+     * @return Closure job.
+     */
+    public PlatformJob createClosureJob(Object task, long ptr, Object job);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index 9bdc3be..e60fbeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -43,4 +43,9 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public PlatformContext context() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public void awaitStart() throws IgniteCheckedException {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 782db4b..8c48649 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -44,4 +44,11 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Platform context.
      */
     public PlatformContext context();
+
+    /**
+     * Await until platform processor is safe to use.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void awaitStart() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformJob.java
new file mode 100644
index 0000000..2ac7194
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformJob.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.compute;
+
+import org.apache.ignite.compute.*;
+
+/**
+ * Platform closure job interface.
+ */
+public interface PlatformJob extends ComputeJob {
+    /**
+     * Gets native pointer to deployed job.
+     *
+     * @return Pointer.
+     */
+    public long pointer();
+
+    /**
+     * Gets native job.
+     *
+     * @return Native job.
+     */
+    public Object job();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
new file mode 100644
index 0000000..b11dc34
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
@@ -0,0 +1,154 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Base interop job.
+ */
+public abstract class PlatformAbstractJob implements PlatformJob, Externalizable {
+    /** Marker object denoting the job execution result is stored in native platform. */
+    static final Object LOC_JOB_RES = new Object();
+
+    /** Grid name. */
+    @IgniteInstanceResource
+    protected transient Ignite ignite;
+
+    /** Parent task; present only on local job instance. */
+    protected transient PlatformAbstractTask task;
+
+    /** Pointer to job in the native platform. */
+    protected transient long ptr;
+
+    /** Job. */
+    protected Object job;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    protected PlatformAbstractJob() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param task Parent task.
+     * @param ptr Pointer.
+     * @param job Job.
+     */
+    protected PlatformAbstractJob(PlatformAbstractTask task, long ptr, Object job) {
+        this.task = task;
+        this.ptr = ptr;
+        this.job = job;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object execute() {
+        try {
+            PlatformProcessor interopProc = PlatformUtils.platformProcessor(ignite);
+
+            interopProc.awaitStart();
+
+            return execute0(interopProc.context());
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
+     * Internal job execution routine.
+     *
+     * @param ctx Interop processor.
+     * @return Result.
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    protected abstract Object execute0(PlatformContext ctx) throws IgniteCheckedException;
+
+    /**
+     * Create job in native platform if needed.
+     *
+     * @param ctx Context.
+     * @return {@code True} if job was created, {@code false} if this is local job and creation is not necessary.
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    protected boolean createJob(PlatformContext ctx) throws IgniteCheckedException {
+        if (ptr == 0) {
+            try (PlatformMemory mem = ctx.memory().allocate()) {
+                PlatformOutputStream out = mem.output();
+
+                PortableRawWriterEx writer = ctx.writer(out);
+
+                writer.writeObject(job);
+
+                out.synchronize();
+
+                ptr = ctx.gateway().computeJobCreate(mem.pointer());
+            }
+
+            return true;
+        }
+        else
+            return false;
+    }
+
+    /**
+     * Run local job.
+     *
+     * @param ctx Context.
+     * @param cancel Cancel flag.
+     * @return Result.
+     */
+    protected Object runLocal(PlatformContext ctx, boolean cancel) {
+        // Local job, must execute it with respect to possible concurrent task completion.
+        if (task.onJobLock()) {
+            try {
+                ctx.gateway().computeJobExecute(ptr, cancel ? 1 : 0, 0);
+
+                return LOC_JOB_RES;
+            }
+            finally {
+                task.onJobUnlock();
+            }
+        }
+        else
+            // Task has completed concurrently, no need to run the job.
+            return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long pointer() {
+        return ptr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object job() {
+        return job;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/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
new file mode 100644
index 0000000..2556796
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
@@ -0,0 +1,202 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Base class for all interop tasks.
+ */
+public abstract class PlatformAbstractTask implements ComputeTask<Object, Void> {
+    /** Platform context. */
+    protected final PlatformContext ctx;
+
+    /** Pointer to the task in the native platform. */
+    protected final long taskPtr;
+
+    /** Lock for safe access to native pointers. */
+    protected final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /** Done flag. */
+    protected boolean done;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param taskPtr Task pointer.
+     */
+    protected PlatformAbstractTask(PlatformContext ctx, long taskPtr) {
+        this.ctx = ctx;
+        this.taskPtr = taskPtr;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
+        assert rcvd.isEmpty() : "Should not cache result in Java for interop task";
+
+        int plc;
+
+        lock.readLock().lock();
+
+        try {
+            assert !done;
+
+            PlatformAbstractJob job = res.getJob();
+
+            assert job.pointer() != 0;
+
+            Object res0bj = res.getData();
+
+            if (res0bj == PlatformAbstractJob.LOC_JOB_RES)
+                // Processing local job execution result.
+                plc = ctx.gateway().computeTaskJobResult(taskPtr, job.pointer(), 0);
+            else {
+                // Processing remote job execution result or exception.
+                try (PlatformMemory mem = ctx.memory().allocate()) {
+                    PlatformOutputStream out = mem.output();
+
+                    PortableRawWriterEx writer = ctx.writer(out);
+
+                    writer.writeUuid(res.getNode().id());
+                    writer.writeBoolean(res.isCancelled());
+
+                    IgniteException err = res.getException();
+
+                    PlatformUtils.writeInvocationResult(writer, res0bj, err);
+
+                    out.synchronize();
+
+                    plc = ctx.gateway().computeTaskJobResult(taskPtr, job.pointer(), mem.pointer());
+                }
+            }
+
+            ComputeJobResultPolicy plc0 = ComputeJobResultPolicy.fromOrdinal((byte) plc);
+
+            assert plc0 != null : plc;
+
+            return plc0;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Void reduce(List<ComputeJobResult> results) {
+        assert results.isEmpty() : "Should not cache result in java for interop task";
+
+        lock.readLock().lock();
+
+        try {
+            assert !done;
+
+            ctx.gateway().computeTaskReduce(taskPtr);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+
+        return null;
+    }
+
+    /**
+     * Callback invoked when task future is completed and all resources could be safely cleaned up.
+     *
+     * @param e If failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void onDone(Exception e) {
+        lock.writeLock().lock();
+
+        try {
+            assert !done;
+
+            if (e == null)
+                // Normal completion.
+                ctx.gateway().computeTaskComplete(taskPtr, 0);
+            else {
+                PlatformNativeException e0 = X.cause(e, PlatformNativeException.class);
+
+                try (PlatformMemory mem = ctx.memory().allocate()) {
+                    PlatformOutputStream out = mem.output();
+
+                    PortableRawWriterEx writer = ctx.writer(out);
+
+                    if (e0 == null) {
+                        writer.writeBoolean(false);
+                        writer.writeString(e.getClass().getName());
+                        writer.writeString(e.getMessage());
+                    }
+                    else {
+                        writer.writeBoolean(true);
+                        writer.writeObject(e0.cause());
+                    }
+
+                    out.synchronize();
+
+                    ctx.gateway().computeTaskComplete(taskPtr, mem.pointer());
+                }
+            }
+        }
+        finally {
+            // Done flag is set irrespective of any exceptions.
+            done = true;
+
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Callback invoked by job when it wants to lock the task.
+     *
+     * @return {@code} True if task is not completed yet, {@code false} otherwise.
+     */
+    @SuppressWarnings("LockAcquiredButNotSafelyReleased")
+    public boolean onJobLock() {
+        lock.readLock().lock();
+
+        if (done) {
+            lock.readLock().unlock();
+
+            return false;
+        }
+        else
+            return true;
+    }
+
+    /**
+     * Callback invoked by job when task can be unlocked.
+     */
+    public void onJobUnlock() {
+        assert !done;
+
+        lock.readLock().unlock();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingMultiClosureTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingMultiClosureTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingMultiClosureTask.java
new file mode 100644
index 0000000..80e7c7e
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingMultiClosureTask.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.compute;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.resources.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop multi-closure task with node balancing.
+ */
+@ComputeTaskNoResultCache
+public class PlatformBalancingMultiClosureTask extends PlatformAbstractTask {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Jobs. */
+    private Collection<PlatformJob> jobs;
+
+    /** Load balancer. */
+    @SuppressWarnings("UnusedDeclaration")
+    @LoadBalancerResource
+    private ComputeLoadBalancer lb;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param taskPtr Task pointer.
+     */
+    public PlatformBalancingMultiClosureTask(PlatformContext ctx, long taskPtr) {
+        super(ctx, taskPtr);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        assert !F.isEmpty(jobs) : "Jobs emptiness must be checked in native platform.";
+
+        if (!F.isEmpty(subgrid)) {
+            Map<ComputeJob, ClusterNode> map = new HashMap<>(jobs.size(), 1);
+
+            for (PlatformJob job : jobs)
+                map.put(job, lb.getBalancedNode(job, null));
+
+            return map;
+        }
+        else
+            return Collections.emptyMap();
+    }
+
+    /**
+     * @param jobs Jobs.
+     */
+    public void jobs(Collection<PlatformJob> jobs) {
+        this.jobs = jobs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureAffinityTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureAffinityTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureAffinityTask.java
new file mode 100644
index 0000000..1b6f24f
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureAffinityTask.java
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop single-closure task with node balancing.
+ */
+@ComputeTaskNoResultCache
+public class PlatformBalancingSingleClosureAffinityTask extends PlatformAbstractTask {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Job. */
+    private PlatformJob job;
+
+    /** Node, according to affinity. */
+    private ClusterNode node;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param taskPtr Task pointer.
+     */
+    public PlatformBalancingSingleClosureAffinityTask(PlatformContext ctx, long taskPtr) {
+        super(ctx, taskPtr);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        assert job != null : "Job null-check must be performed in native platform.";
+
+        return Collections.singletonMap(job, node);
+    }
+
+    /**
+     * @param job Job.
+     */
+    public void job(PlatformJob job) {
+        this.job = job;
+    }
+
+    /**
+     * Init affinity.
+     *
+     * @param cacheName Cache name.
+     * @param affKey Affinity key.
+     * @param ctx Kernal context.
+     */
+    public void affinity(String cacheName, Object affKey, GridKernalContext ctx) {
+        try {
+            final Object affKey0 = ctx.affinity().affinityKey(cacheName, affKey);
+
+            node = ctx.affinity().mapKeyToNode(cacheName, affKey0);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureTask.java
new file mode 100644
index 0000000..32b9464
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBalancingSingleClosureTask.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.compute;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.resources.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop single-closure task with node balancing.
+ */
+@ComputeTaskNoResultCache
+public class PlatformBalancingSingleClosureTask extends PlatformAbstractTask {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Jobs. */
+    private PlatformJob job;
+
+    /** Load balancer. */
+    @SuppressWarnings("UnusedDeclaration")
+    @LoadBalancerResource
+    private ComputeLoadBalancer lb;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param taskPtr Task pointer.
+     */
+    public PlatformBalancingSingleClosureTask(PlatformContext ctx, long taskPtr) {
+        super(ctx, taskPtr);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        assert job != null : "Job null-check must be performed in native platform.";
+
+        if (!F.isEmpty(subgrid)) {
+            Map<ComputeJob, ClusterNode> map = new HashMap<>(1, 1);
+
+            map.put(job, lb.getBalancedNode(job, null));
+
+            return map;
+        }
+        else
+            return Collections.emptyMap();
+    }
+
+    /**
+     * @param job Job.
+     */
+    public void job(PlatformJob job) {
+        this.job = job;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingMultiClosureTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingMultiClosureTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingMultiClosureTask.java
new file mode 100644
index 0000000..64328b5
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingMultiClosureTask.java
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop multi-closure task with broadcast semantics.
+ */
+@ComputeTaskNoResultCache
+public class PlatformBroadcastingMultiClosureTask extends PlatformAbstractTask {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Jobs. */
+    private Collection<PlatformJob> jobs;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param taskPtr Task pointer.
+     */
+    public PlatformBroadcastingMultiClosureTask(PlatformContext ctx, long taskPtr) {
+        super(ctx, taskPtr);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        assert !F.isEmpty(jobs) : "Jobs emptiness must be checked in native platform.";
+
+        if (!F.isEmpty(subgrid)) {
+            Map<ComputeJob, ClusterNode> map = new HashMap<>(jobs.size() * subgrid.size(), 1);
+
+            for (PlatformJob job : jobs) {
+                boolean first = true;
+
+                for (ClusterNode node : subgrid) {
+                    if (first) {
+                        map.put(job, node);
+
+                        first = false;
+                    }
+                    else
+                        map.put(ctx.createClosureJob(this, job.pointer(), job.job()), node);
+                }
+            }
+
+            return map;
+        }
+        else
+            return Collections.emptyMap();
+    }
+
+    /**
+     * @param jobs Jobs.
+     */
+    public void jobs(Collection<PlatformJob> jobs) {
+        this.jobs = jobs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingSingleClosureTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingSingleClosureTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingSingleClosureTask.java
new file mode 100644
index 0000000..fa36920
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformBroadcastingSingleClosureTask.java
@@ -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.
+ */
+
+package org.apache.ignite.internal.processors.platform.compute;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop single-closure task with broadcast semantics.
+ */
+@ComputeTaskNoResultCache
+public class PlatformBroadcastingSingleClosureTask extends PlatformAbstractTask {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private PlatformJob job;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param taskPtr Task pointer.
+     */
+    public PlatformBroadcastingSingleClosureTask(PlatformContext ctx, long taskPtr) {
+        super(ctx, taskPtr);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        assert job != null : "Job null-check must be performed in native platform.";
+
+        if (!F.isEmpty(subgrid)) {
+            Map<ComputeJob, ClusterNode> map = new HashMap<>(subgrid.size(), 1);
+
+            boolean first = true;
+
+            for (ClusterNode node : subgrid) {
+                if (first) {
+                    map.put(job, node);
+
+                    first = false;
+                }
+                else
+                    map.put(ctx.createClosureJob(this, job.pointer(), job.job()), node);
+            }
+
+            return map;
+        }
+        else
+            return Collections.emptyMap();
+    }
+
+    /**
+     * @param job Job.
+     */
+    public void job(PlatformJob job) {
+        this.job = job;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
new file mode 100644
index 0000000..5bb8433
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
@@ -0,0 +1,102 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Light-weight interop job. Comparing to regular job, this guy has simpler logic because we should not
+ * bother with delayed serialization and cancellation.
+ */
+public class PlatformClosureJob extends PlatformAbstractJob {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    public PlatformClosureJob() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param task Parent task.
+     * @param ptr Job pointer.
+     * @param job Job.
+     */
+    public PlatformClosureJob(PlatformAbstractTask task, long ptr, Object job) {
+        super(task, ptr, job);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object execute0(PlatformContext ctx) throws IgniteCheckedException {
+        if (task == null) {
+            // Remote job execution.
+            assert ptr == 0;
+
+            createJob(ctx);
+
+            try (PlatformMemory mem = ctx.memory().allocate()) {
+                PlatformInputStream in = mem.input();
+
+                ctx.gateway().computeJobExecute(ptr, 0, mem.pointer());
+
+                in.synchronize();
+
+                PortableRawReaderEx reader = ctx.reader(in);
+
+                return PlatformUtils.readInvocationResult(ctx, reader);
+            }
+            finally {
+                ctx.gateway().computeJobDestroy(ptr);
+            }
+        }
+        else {
+            // Local job execution.
+            assert ptr != 0;
+
+            return runLocal(ctx, false);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        assert job != null;
+
+        out.writeObject(job);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        job = in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
new file mode 100644
index 0000000..2b1f6be
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -0,0 +1,323 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.portable.*;
+
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.*;
+
+/**
+ * Interop compute.
+ */
+@SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored", "UnusedDeclaration"})
+public class PlatformCompute extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_AFFINITY = 1;
+
+    /** */
+    private static final int OP_BROADCAST = 2;
+
+    /** */
+    private static final int OP_EXEC = 3;
+
+    /** */
+    private static final int OP_EXEC_ASYNC = 4;
+
+    /** */
+    private static final int OP_UNICAST = 5;
+
+    /** Compute instance. */
+    private final IgniteComputeImpl compute;
+
+    /** Future for previous asynchronous operation. */
+    protected ThreadLocal<IgniteFuture<?>> curFut = new ThreadLocal<>();
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param compute Compute instance.
+     */
+    public PlatformCompute(PlatformContext platformCtx, IgniteComputeImpl compute) {
+        super(platformCtx);
+
+        this.compute = compute;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int processInOp(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+        switch (type) {
+            case OP_UNICAST:
+                processClosures(reader.readLong(), reader, false, false);
+
+                return TRUE;
+
+            case OP_BROADCAST:
+                processClosures(reader.readLong(), reader, true, false);
+
+                return TRUE;
+
+            case OP_AFFINITY:
+                processClosures(reader.readLong(), reader, false, true);
+
+                return TRUE;
+
+            default:
+                return throwUnsupported(type);
+        }
+    }
+
+    /**
+     * Process closure execution request.
+     *
+     * @param taskPtr Task pointer.
+     * @param reader Reader.
+     * @param broadcast broadcast flag.
+     */
+    private void processClosures(long taskPtr, PortableRawReaderEx reader, boolean broadcast, boolean affinity) {
+        PlatformAbstractTask task;
+
+        int size = reader.readInt();
+
+        if (size == 1) {
+            if (broadcast) {
+                PlatformBroadcastingSingleClosureTask task0 =
+                    new PlatformBroadcastingSingleClosureTask(platformCtx, taskPtr);
+
+                task0.job(nextClosureJob(task0, reader));
+
+                task = task0;
+            }
+            else if (affinity) {
+                PlatformBalancingSingleClosureAffinityTask task0 =
+                    new PlatformBalancingSingleClosureAffinityTask(platformCtx, taskPtr);
+
+                task0.job(nextClosureJob(task0, reader));
+
+                task0.affinity(reader.readString(), reader.readObjectDetached(), platformCtx.kernalContext());
+
+                task = task0;
+            }
+            else {
+                PlatformBalancingSingleClosureTask task0 = new PlatformBalancingSingleClosureTask(platformCtx, taskPtr);
+
+                task0.job(nextClosureJob(task0, reader));
+
+                task = task0;
+            }
+        }
+        else {
+            if (broadcast)
+                task = new PlatformBroadcastingMultiClosureTask(platformCtx, taskPtr);
+            else
+                task = new PlatformBalancingMultiClosureTask(platformCtx, taskPtr);
+
+            Collection<PlatformJob> jobs = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                jobs.add(nextClosureJob(task, reader));
+
+            if (broadcast)
+                ((PlatformBroadcastingMultiClosureTask)task).jobs(jobs);
+            else
+                ((PlatformBalancingMultiClosureTask)task).jobs(jobs);
+        }
+
+        platformCtx.kernalContext().task().setThreadContext(TC_SUBGRID, compute.clusterGroup().nodes());
+
+        executeNative0(task);
+    }
+
+    /**
+     * Read the next closure job from the reader.
+     *
+     * @param task Task.
+     * @param reader Reader.
+     * @return Closure job.
+     */
+    private PlatformJob nextClosureJob(PlatformAbstractTask task, PortableRawReaderEx reader) {
+        return platformCtx.createClosureJob(task, reader.readLong(), reader.readObjectDetached());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInOutOp(int type, PortableRawReaderEx reader, PortableRawWriterEx writer,
+        Object arg) throws IgniteCheckedException {
+        switch (type) {
+            case OP_EXEC:
+                writer.writeObjectDetached(executeJavaTask(reader, false));
+
+                break;
+
+            case OP_EXEC_ASYNC:
+                writer.writeObjectDetached(executeJavaTask(reader, true));
+
+                break;
+            default:
+                throwUnsupported(type);
+        }
+    }
+
+    /**
+     * Execute native full-fledged task.
+     *
+     * @param taskPtr Pointer to the task.
+     * @param topVer Topology version.
+     */
+    public void executeNative(long taskPtr, long topVer) {
+        final PlatformFullTask task = new PlatformFullTask(platformCtx, compute, taskPtr, topVer);
+
+        executeNative0(task);
+    }
+
+    /**
+     * Set "withTimeout" state.
+     *
+     * @param timeout Timeout (milliseconds).
+     */
+    public void withTimeout(long timeout) {
+        compute.withTimeout(timeout);
+    }
+
+    /**
+     * Set "withNoFailover" state.
+     */
+    public void withNoFailover() {
+        compute.withNoFailover();
+    }
+
+    /** <inheritDoc /> */
+    @Override protected IgniteFuture currentFuture() throws IgniteCheckedException {
+        IgniteFuture<?> fut = curFut.get();
+
+        if (fut == null)
+            throw new IllegalStateException("Asynchronous operation not started.");
+
+        return fut;
+    }
+
+    /**
+     * Execute task.
+     *
+     * @param task Task.
+     */
+    private void executeNative0(final PlatformAbstractTask task) {
+        IgniteInternalFuture fut = compute.executeAsync(task, null);
+
+        fut.listen(new IgniteInClosure<IgniteInternalFuture>() {
+            private static final long serialVersionUID = 0L;
+
+            @Override public void apply(IgniteInternalFuture fut) {
+                try {
+                    fut.get();
+
+                    task.onDone(null);
+                }
+                catch (IgniteCheckedException e) {
+                    task.onDone(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * Execute task taking arguments from the given reader.
+     *
+     * @param reader Reader.
+     * @return Task result.
+     */
+    protected Object executeJavaTask(PortableRawReaderEx reader, boolean async) {
+        String taskName = reader.readString();
+        boolean keepPortable = reader.readBoolean();
+        Object arg = reader.readObjectDetached();
+
+        Collection<UUID> nodeIds = readNodeIds(reader);
+
+        IgniteCompute compute0 = computeForTask(nodeIds);
+
+        if (async)
+            compute0 = compute0.withAsync();
+
+        if (!keepPortable && arg instanceof PortableObjectImpl)
+            arg = ((PortableObject)arg).deserialize();
+
+        Object res = compute0.execute(taskName, arg);
+
+        if (async) {
+            curFut.set(compute0.future().chain(new C1<IgniteFuture, Object>() {
+                private static final long serialVersionUID = 0L;
+
+                @Override public Object apply(IgniteFuture fut) {
+                    return toPortable(fut.get());
+                }
+            }));
+
+            return null;
+        }
+        else
+            return toPortable(res);
+    }
+
+    /**
+     * Convert object to portable form.
+     *
+     * @param src Source object.
+     * @return Result.
+     */
+    private Object toPortable(Object src) {
+        return platformCtx.kernalContext().grid().portables().toPortable(src);
+    }
+
+    /**
+     * Read node IDs.
+     *
+     * @param reader Reader.
+     * @return Node IDs.
+     */
+    protected Collection<UUID> readNodeIds(PortableRawReaderEx reader) {
+        if (reader.readBoolean()) {
+            int len = reader.readInt();
+
+            List<UUID> res = new ArrayList<>(len);
+
+            for (int i = 0; i < len; i++)
+                res.add(reader.readUuid());
+
+            return res;
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Get compute object for the given node IDs.
+     *
+     * @param nodeIds Node IDs.
+     * @return Compute object.
+     */
+    protected IgniteCompute computeForTask(Collection<UUID> nodeIds) {
+        return nodeIds == null ? compute :
+            platformCtx.kernalContext().grid().compute(compute.clusterGroup().forNodeIds(nodeIds));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
new file mode 100644
index 0000000..b364409
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
@@ -0,0 +1,217 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.processors.platform.utils.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Wrapper around job created in native platform.
+ * <p>
+ * If the job is expected to be executed locally, it contains only pointer to the corresponding entity in the native
+ * platform. In case of topology change or failover, job is serialized on demand.
+ * <p>
+ * If we know in advance that the job is to be executed on remote node, then it is serialized into byte array right
+ * away.
+ * <p>
+ * This class is not thread safe.
+ */
+@SuppressWarnings({"FieldCanBeLocal"})
+public class PlatformFullJob extends PlatformAbstractJob {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Job is initialized. */
+    private static final byte STATE_INIT = 0;
+
+    /** Job is running. */
+    private static final byte STATE_RUNNING = 1;
+
+    /** Job execution completed. */
+    private static final byte STATE_COMPLETED = 2;
+
+    /** Job cancelled. */
+    private static final byte STATE_CANCELLED = 3;
+
+    /** Platform context. */
+    private transient PlatformContext ctx;
+
+    /** Serialized job. */
+    private transient byte state;
+
+    /**
+     * {@link java.io.Externalizable} support.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public PlatformFullJob() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param task Parent task.
+     * @param ptr Job pointer.
+     * @param job Job.
+     */
+    public PlatformFullJob(PlatformContext ctx, PlatformAbstractTask task, long ptr, Object job) {
+        super(task, ptr, job);
+
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object execute0(PlatformContext ctx) throws IgniteCheckedException {
+        boolean cancel = false;
+
+        synchronized (this) {
+            // 1. Create job if necessary.
+            if (task == null) {
+                assert ptr == 0;
+
+                createJob(ctx);
+            }
+            else
+                assert ptr != 0;
+
+            // 2. Set correct state.
+            if (state == STATE_INIT)
+                state = STATE_RUNNING;
+            else {
+                assert state == STATE_CANCELLED;
+
+                cancel = true;
+            }
+        }
+
+        try {
+            if (task != null)
+                return runLocal(ctx, cancel);
+            else {
+                try (PlatformMemory mem = ctx.memory().allocate()) {
+                    PlatformInputStream in = mem.input();
+
+                    ctx.gateway().computeJobExecute(ptr, cancel ? 1 : 0, mem.pointer());
+
+                    in.synchronize();
+
+                    PortableRawReaderEx reader = ctx.reader(in);
+
+                    return PlatformUtils.readInvocationResult(ctx, reader);
+                }
+            }
+        }
+        finally {
+            synchronized (this) {
+                if (task == null) {
+                    assert ptr != 0;
+
+                    ctx.gateway().computeJobDestroy(ptr);
+                }
+
+                if (state == STATE_RUNNING)
+                    state = STATE_COMPLETED;
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        PlatformProcessor proc = PlatformUtils.platformProcessor(ignite);
+
+        synchronized (this) {
+            if (state == STATE_INIT)
+                state = STATE_CANCELLED;
+            else if (state == STATE_RUNNING) {
+                assert ptr != 0;
+
+                try {
+                    proc.context().gateway().computeJobCancel(ptr);
+                }
+                finally {
+                    state = STATE_CANCELLED;
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        if (job == null) {
+            assert ptr != 0;
+
+            try {
+                if (task != null) {
+                    if (task.onJobLock()) {
+                        try {
+                            serialize();
+                        }
+                        finally {
+                            task.onJobUnlock();
+                        }
+                    }
+                    else
+                        throw new IgniteCheckedException("Task already completed: " + task);
+                }
+                else
+                    serialize();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IOException("Failed to serialize interop job.", e);
+            }
+        }
+
+        assert job != null;
+
+        out.writeObject(job);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        job = in.readObject();
+    }
+
+    /**
+     * Internal job serialization routine.
+     *
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    private void serialize() throws IgniteCheckedException {
+        try (PlatformMemory mem = ctx.memory().allocate()) {
+            PlatformInputStream in = mem.input();
+
+            boolean res = ctx.gateway().computeJobSerialize(ptr, mem.pointer()) == 1;
+
+            in.synchronize();
+
+            PortableRawReaderEx reader = ctx.reader(in);
+
+            if (res)
+                job = reader.readObjectDetached();
+            else
+                throw new IgniteCheckedException(reader.readString());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
new file mode 100644
index 0000000..7777143
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
@@ -0,0 +1,185 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.memory.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Interop task which requires full execution cycle.
+ */
+@ComputeTaskNoResultCache
+public final class PlatformFullTask extends PlatformAbstractTask {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Initial topology version. */
+    private final long topVer;
+
+    /** Compute instance. */
+    private final IgniteComputeImpl compute;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Platform context.
+     * @param compute Target compute instance.
+     * @param taskPtr Pointer to the task in the native platform.
+     * @param topVer Initial topology version.
+     */
+    public PlatformFullTask(PlatformContext ctx, IgniteComputeImpl compute, long taskPtr, long topVer) {
+        super(ctx, taskPtr);
+
+        this.compute = compute;
+        this.topVer = topVer;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable Object arg) {
+        assert arg == null;
+
+        lock.readLock().lock();
+
+        try {
+            assert !done;
+
+            Collection<ClusterNode> nodes = compute.clusterGroup().nodes();
+
+            PlatformMemoryManager memMgr = ctx.memory();
+
+            try (PlatformMemory outMem = memMgr.allocate()) {
+                PlatformOutputStream out = outMem.output();
+
+                PortableRawWriterEx writer = ctx.writer(out);
+
+                write(writer, nodes, subgrid);
+
+                out.synchronize();
+
+                try (PlatformMemory inMem = memMgr.allocate()) {
+                    PlatformInputStream in = inMem.input();
+
+                    ctx.gateway().computeTaskMap(taskPtr, outMem.pointer(), inMem.pointer());
+
+                    in.synchronize();
+
+                    PortableRawReaderEx reader = ctx.reader(in);
+
+                    return read(reader, nodes);
+                }
+            }
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Write topology information.
+     *
+     * @param writer Writer.
+     * @param nodes Current topology nodes.
+     * @param subgrid Subgrid.
+     */
+    private void write(PortableRawWriterEx writer, Collection<ClusterNode> nodes, List<ClusterNode> subgrid) {
+        GridDiscoveryManager discoMgr = ctx.kernalContext().discovery();
+
+        long curTopVer = discoMgr.topologyVersion();
+
+        if (topVer != curTopVer) {
+            writer.writeBoolean(true);
+
+            writer.writeLong(curTopVer);
+
+            writer.writeInt(nodes.size());
+
+            // Write subgrid size for more precise collection allocation on native side.
+            writer.writeInt(subgrid.size());
+
+            for (ClusterNode node : nodes) {
+                ctx.writeNode(writer, node);
+                writer.writeBoolean(subgrid.contains(node));
+            }
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /**
+     * Read map result.
+     *
+     * @param reader Reader.
+     * @param nodes Current topology nodes.
+     * @return Map result.
+     */
+    private Map<ComputeJob, ClusterNode> read(PortableRawReaderEx reader, Collection<ClusterNode> nodes) {
+        if (reader.readBoolean()) {
+            if (!reader.readBoolean())
+                return null;
+
+            int size = reader.readInt();
+
+            Map<ComputeJob, ClusterNode> map = U.newHashMap(size);
+
+            for (int i = 0; i < size; i++) {
+                long ptr = reader.readLong();
+
+                Object nativeJob = reader.readBoolean() ? reader.readObjectDetached() : null;
+
+                PlatformJob job = ctx.createJob(this, ptr, nativeJob);
+
+                UUID jobNodeId = reader.readUuid();
+
+                assert jobNodeId != null;
+
+                ClusterNode jobNode = ctx.kernalContext().discovery().node(jobNodeId);
+
+                if (jobNode == null) {
+                    // Special case when node has left the grid at this point.
+                    // We expect task processor to perform necessary failover.
+                    for (ClusterNode node : nodes) {
+                        if (node.id().equals(jobNodeId)) {
+                            jobNode = node;
+
+                            break;
+                        }
+                    }
+
+                    assert jobNode != null;
+                }
+
+                map.put(job, jobNode);
+            }
+
+            return map;
+        }
+        else
+            throw new IgniteException(reader.readString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/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
new file mode 100644
index 0000000..e77f5d8
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformNativeException.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * 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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5bbb8a33/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 0777f9a..7dad4b7 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
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.portable.*;
 import org.apache.ignite.internal.processors.platform.*;
+import org.apache.ignite.internal.processors.platform.compute.*;
 import org.apache.ignite.internal.processors.platform.memory.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -655,6 +656,80 @@ public class PlatformUtils {
     }
 
     /**
+     * Writes invocation result (of a job/service/etc) using a common protocol.
+     *
+     * @param writer Writer.
+     * @param resObj Result.
+     * @param err Error.
+     */
+    public static void writeInvocationResult(PortableRawWriterEx writer, Object resObj, Exception err)
+    {
+        if (err == null) {
+            writer.writeBoolean(true);
+            writer.writeObject(resObj);
+        }
+        else {
+            writer.writeBoolean(false);
+
+            PlatformNativeException nativeErr = null;
+
+            if (err instanceof IgniteCheckedException)
+                nativeErr = ((IgniteCheckedException)err).getCause(PlatformNativeException.class);
+            else if (err instanceof IgniteException)
+                nativeErr = ((IgniteException)err).getCause(PlatformNativeException.class);
+
+            if (nativeErr == null) {
+                writer.writeBoolean(false);
+                writer.writeString(err.getClass().getName());
+                writer.writeString(err.getMessage());
+            }
+            else {
+                writer.writeBoolean(true);
+                writer.writeObject(nativeErr.cause());
+            }
+        }
+    }
+
+    /**
+     * Reads invocation result (of a job/service/etc) using a common protocol.
+     *
+     * @param ctx Platform context.
+     * @param reader Reader.
+     * @return Result.
+     * @throws IgniteCheckedException When invocation result is an error.
+     */
+    public static Object readInvocationResult(PlatformContext ctx, PortableRawReaderEx reader)
+        throws IgniteCheckedException {
+        // 1. Read success flag.
+        boolean success = reader.readBoolean();
+
+        if (success)
+            // 2. Return result as is.
+            return reader.readObjectDetached();
+        else {
+            // 3. Read whether exception is in form of object or string.
+            boolean hasException = reader.readBoolean();
+
+            if (hasException) {
+                // 4. Full exception.
+                Object nativeErr = reader.readObjectDetached();
+
+                assert nativeErr != null;
+
+                throw ctx.createNativeException(nativeErr);
+            }
+            else {
+                // 5. Native exception was not serializable, we have only message.
+                String errMsg = reader.readString();
+
+                assert errMsg != null;
+
+                throw new IgniteCheckedException(errMsg);
+            }
+        }
+    }
+
+    /**
      * Private constructor.
      */
     private PlatformUtils() {


[04/50] [abbrv] ignite git commit: Platforms: minor refactoring: interopCtx -> platformCtx.

Posted by ak...@apache.org.
Platforms: minor refactoring: interopCtx -> platformCtx.


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

Branch: refs/heads/ignite-843
Commit: d9a13974b6e7c5fdd2b533e806e928fcca33bbaa
Parents: 8df6b93
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 16:03:25 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 16:03:25 2015 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        | 44 ++++++++++----------
 .../query/PlatformAbstractQueryCursor.java      |  6 +--
 .../cache/query/PlatformFieldsQueryCursor.java  |  6 +--
 .../cache/query/PlatformQueryCursor.java        |  6 +--
 .../transactions/PlatformTransactions.java      | 12 +++---
 .../platform/utils/PlatformFutureUtils.java     | 12 +++---
 .../platform/utils/PlatformUtils.java           |  4 +-
 7 files changed, 45 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/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 ec728e4..b68b16e 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
@@ -37,8 +37,8 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** */
     private static final int OP_META = -1;
 
-    /** Interop context. */
-    protected final PlatformContext interopCtx;
+    /** Context. */
+    protected final PlatformContext platformCtx;
 
     /** Logger. */
     protected final IgniteLogger log;
@@ -46,21 +46,21 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /**
      * Constructor.
      *
-     * @param interopCtx Interop context.
+     * @param platformCtx Context.
      */
-    protected PlatformAbstractTarget(PlatformContext interopCtx) {
-        this.interopCtx = interopCtx;
+    protected PlatformAbstractTarget(PlatformContext platformCtx) {
+        this.platformCtx = platformCtx;
 
-        log = interopCtx.kernalContext().log(PlatformAbstractTarget.class);
+        log = platformCtx.kernalContext().log(PlatformAbstractTarget.class);
     }
 
     /** {@inheritDoc} */
     @Override public int inOp(int type, long memPtr) throws Exception {
-        try (PlatformMemory mem = interopCtx.memory().get(memPtr)) {
-            PortableRawReaderEx reader = interopCtx.reader(mem);
+        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
+            PortableRawReaderEx reader = platformCtx.reader(mem);
 
             if (type == OP_META) {
-                interopCtx.processMetadata(reader);
+                platformCtx.processMetadata(reader);
 
                 return TRUE;
             }
@@ -74,8 +74,8 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
 
     /** {@inheritDoc} */
     @Override public Object inOpObject(int type, long memPtr) throws Exception {
-        try (PlatformMemory mem = interopCtx.memory().get(memPtr)) {
-            PortableRawReaderEx reader = interopCtx.reader(mem);
+        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
+            PortableRawReaderEx reader = platformCtx.reader(mem);
 
             return processInOpObject(type, reader);
         }
@@ -86,10 +86,10 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
 
     /** {@inheritDoc} */
     @Override public void outOp(int type, long memPtr) throws Exception {
-        try (PlatformMemory mem = interopCtx.memory().get(memPtr)) {
+        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = interopCtx.writer(out);
+            PortableRawWriterEx writer = platformCtx.writer(out);
 
             processOutOp(type, writer);
 
@@ -107,13 +107,13 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
 
     /** {@inheritDoc} */
     @Override public void inOutOp(int type, long inMemPtr, long outMemPtr, Object arg) throws Exception {
-        try (PlatformMemory inMem = interopCtx.memory().get(inMemPtr)) {
-            PortableRawReaderEx reader = interopCtx.reader(inMem);
+        try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
+            PortableRawReaderEx reader = platformCtx.reader(inMem);
 
-            try (PlatformMemory outMem = interopCtx.memory().get(outMemPtr)) {
+            try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
                 PlatformOutputStream out = outMem.output();
 
-                PortableRawWriterEx writer = interopCtx.writer(out);
+                PortableRawWriterEx writer = platformCtx.writer(out);
 
                 processInOutOp(type, reader, writer, arg);
 
@@ -136,10 +136,10 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /**
-     * @return Interop context.
+     * @return Context.
      */
-    public PlatformContext interopContext() {
-        return interopCtx;
+    public PlatformContext platformContext() {
+        return platformCtx;
     }
 
     /**
@@ -150,7 +150,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      */
     @SuppressWarnings("UnusedDeclaration")
     public void listenFuture(final long futId, int typ) throws IgniteCheckedException {
-        PlatformFutureUtils.listen(interopCtx, currentFutureWrapped(), futId, typ, null);
+        PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, null);
     }
 
     /**
@@ -162,7 +162,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      */
     @SuppressWarnings("UnusedDeclaration")
     public void listenFuture(final long futId, int typ, int opId) throws IgniteCheckedException {
-        PlatformFutureUtils.listen(interopCtx, currentFutureWrapped(), futId, typ, futureWriter(opId));
+        PlatformFutureUtils.listen(platformCtx, currentFutureWrapped(), futId, typ, futureWriter(opId));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
index cdd29fd..c01da0e 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
@@ -50,12 +50,12 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
     /**
      * Constructor.
      *
-     * @param interopCtx Interop context.
+     * @param platformCtx Context.
      * @param cursor Underlying cursor.
      * @param batchSize Batch size.
      */
-    public PlatformAbstractQueryCursor(PlatformContext interopCtx, QueryCursorEx<T> cursor, int batchSize) {
-        super(interopCtx);
+    public PlatformAbstractQueryCursor(PlatformContext platformCtx, QueryCursorEx<T> cursor, int batchSize) {
+        super(platformCtx);
 
         this.cursor = cursor;
         this.batchSize = batchSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
index f18a79a..6b2902c 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
@@ -30,12 +30,12 @@ public class PlatformFieldsQueryCursor extends PlatformAbstractQueryCursor<List<
     /**
      * Constructor.
      *
-     * @param interopCtx Interop context.
+     * @param platformCtx Platform context.
      * @param cursor Backing cursor.
      * @param batchSize Batch size.
      */
-    public PlatformFieldsQueryCursor(PlatformContext interopCtx, QueryCursorEx<List<?>> cursor, int batchSize) {
-        super(interopCtx, cursor, batchSize);
+    public PlatformFieldsQueryCursor(PlatformContext platformCtx, QueryCursorEx<List<?>> cursor, int batchSize) {
+        super(platformCtx, cursor, batchSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
index cc96d6f..0d323a3 100644
--- a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
@@ -30,12 +30,12 @@ public class PlatformQueryCursor extends PlatformAbstractQueryCursor<Cache.Entry
     /**
      * Constructor.
      *
-     * @param interopCtx Interop context.
+     * @param platformCtx Context.
      * @param cursor Backing cursor.
      * @param batchSize Batch size.
      */
-    public PlatformQueryCursor(PlatformContext interopCtx, QueryCursorEx<Cache.Entry> cursor, int batchSize) {
-        super(interopCtx, cursor, batchSize);
+    public PlatformQueryCursor(PlatformContext platformCtx, QueryCursorEx<Cache.Entry> cursor, int batchSize) {
+        super(platformCtx, cursor, batchSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/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 fa63840..aedc380 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
@@ -54,12 +54,12 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     /**
      * Constructor.
      *
-     * @param interopCtx Interop context.
+     * @param platformCtx Context.
      */
-    public PlatformTransactions(PlatformContext interopCtx) {
-        super(interopCtx);
+    public PlatformTransactions(PlatformContext platformCtx) {
+        super(platformCtx);
 
-        txs = interopCtx.kernalContext().grid().transactions();
+        txs = platformCtx.kernalContext().grid().transactions();
     }
 
     /**
@@ -175,7 +175,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
             }
         });
 
-        PlatformFutureUtils.listen(interopCtx, fut, futId, PlatformFutureUtils.TYP_OBJ);
+        PlatformFutureUtils.listen(platformCtx, fut, futId, PlatformFutureUtils.TYP_OBJ);
     }
 
     /**
@@ -230,7 +230,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_CACHE_CONFIG_PARAMETERS:
-                TransactionConfiguration txCfg = interopCtx.kernalContext().config().getTransactionConfiguration();
+                TransactionConfiguration txCfg = platformCtx.kernalContext().config().getTransactionConfiguration();
 
                 writer.writeEnum(txCfg.getDefaultTxConcurrency());
                 writer.writeEnum(txCfg.getDefaultTxIsolation());

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/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 fa986fe..677cb1c 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
@@ -58,7 +58,7 @@ public class PlatformFutureUtils {
     /**
      * Listen future.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param fut Java future.
      * @param futPtr Native future pointer.
      * @param typ Expected return type.
@@ -70,7 +70,7 @@ public class PlatformFutureUtils {
     /**
      * Listen future.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param fut Java future.
      * @param futPtr Native future pointer.
      * @param typ Expected return type.
@@ -84,7 +84,7 @@ public class PlatformFutureUtils {
     /**
      * Listen future.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param fut Java future.
      * @param futPtr Native future pointer.
      * @param writer Writer.
@@ -96,7 +96,7 @@ public class PlatformFutureUtils {
     /**
      * Listen future.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param listenable Listenable entry.
      * @param futPtr Native future pointer.
      * @param typ Expected return type.
@@ -200,7 +200,7 @@ public class PlatformFutureUtils {
     /**
      * Write future error.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param futPtr Future pointer.
      * @param err Error.
      */
@@ -226,7 +226,7 @@ public class PlatformFutureUtils {
      *
      * @param obj Object to write.
      * @param err Error to write.
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param writer Writer.
      * @param futPtr Future pointer.
      * @return Value indicating whether custom write was performed. When false, default write will be used.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9a13974/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 614346a..66e87e2 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
@@ -434,7 +434,7 @@ public class PlatformUtils {
     /**
      * Apply continuous query events to listener.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param lsnrPtr Listener pointer.
      * @param evts Events.
      * @throws javax.cache.event.CacheEntryListenerException In case of failure.
@@ -473,7 +473,7 @@ public class PlatformUtils {
     /**
      * Evaluate the filter.
      *
-     * @param ctx Interop context.
+     * @param ctx Context.
      * @param filterPtr Native filter pointer.
      * @param evt Event.
      * @return Result.


[31/50] [abbrv] ignite git commit: ignite-1307 Do not complete prepare future with null result

Posted by ak...@apache.org.
ignite-1307 Do not complete prepare future with null result


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

Branch: refs/heads/ignite-843
Commit: b132006f85df6c1b15e676dd0d8082cf44984b84
Parents: 3d46b62
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 13:39:26 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 13:39:26 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtTxPrepareFuture.java          | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b132006f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 2071f8e..4c39476 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -729,7 +729,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * Completes this future.
      */
     public void complete() {
-        onComplete(null);
+        GridNearTxPrepareResponse res = new GridNearTxPrepareResponse();
+
+        res.error(new IgniteCheckedException("Failed to prepare transaction."));
+
+        onComplete(res);
     }
 
     /**


[34/50] [abbrv] ignite git commit: Added failover tests for async operations.

Posted by ak...@apache.org.
Added failover tests for async operations.


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

Branch: refs/heads/ignite-843
Commit: 73ab5e2f7bc121eaf496096f205547c026c91464
Parents: b132006
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 15:07:49 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 15:07:49 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +
 ...acheAsyncOperationsFailoverAbstractTest.java | 329 +++++++++++++++++++
 .../CacheAsyncOperationsFailoverAtomicTest.java |  32 ++
 .../CacheAsyncOperationsFailoverTxTest.java     |  32 ++
 .../IgniteCacheFailoverTestSuite2.java          |   4 +
 6 files changed, 405 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/73ab5e2f/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 54d33e0..c3bbbe4 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
@@ -4135,6 +4135,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 IgniteInternalFuture<T> f = new GridEmbeddedFuture(fut,
                     new IgniteOutClosure<IgniteInternalFuture>() {
                         @Override public IgniteInternalFuture<T> apply() {
+                            if (ctx.kernalContext().isStopping())
+                                return new GridFinishedFuture<>(
+                                    new IgniteCheckedException("Operation has been cancelled (node is stopping)."));
+
                             return op.op(tx0).chain(new CX1<IgniteInternalFuture<T>, T>() {
                                 @Override public T applyx(IgniteInternalFuture<T> tFut) throws IgniteCheckedException {
                                     try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/73ab5e2f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 9087d20..4b8585e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -570,6 +570,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 IgniteInternalFuture<T> f = new GridEmbeddedFuture(fut,
                     new IgniteOutClosure<IgniteInternalFuture>() {
                         @Override public IgniteInternalFuture<T> apply() {
+                            if (ctx.kernalContext().isStopping())
+                                return new GridFinishedFuture<>(
+                                    new IgniteCheckedException("Operation has been cancelled (node is stopping)."));
+
                             return op.apply();
                         }
                     });

http://git-wip-us.apache.org/repos/asf/ignite/blob/73ab5e2f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
new file mode 100644
index 0000000..1669404
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
@@ -0,0 +1,329 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public abstract class CacheAsyncOperationsFailoverAbstractTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int NODE_CNT = 4;
+
+    /** */
+    private static final long TEST_TIME = 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} */
+    @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 testPutAllAsyncFailover() throws Exception {
+        putAllAsyncFailover(5, 10);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllAsyncFailoverManyThreads() throws Exception {
+        putAllAsyncFailover(ignite(0).configuration().getSystemThreadPoolSize() * 2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAsyncFailover() throws Exception {
+        for (int i = 0; i < 3; i++) {
+            log.info("Iteration: " + i);
+
+            startGrid(NODE_CNT);
+
+            final IgniteCache<TestKey, TestValue> cache = ignite(0).<TestKey, TestValue>cache(null).withAsync();
+
+            int ops = cache.getConfiguration(CacheConfiguration.class).getMaxConcurrentAsyncOperations();
+
+            log.info("Max concurrent async operations: " + ops);
+
+            assertTrue(ops > 0);
+
+            final List<IgniteFuture<?>> futs = Collections.synchronizedList(new ArrayList<IgniteFuture<?>>(ops));
+
+            final AtomicInteger left = new AtomicInteger(ops);
+
+            GridTestUtils.runMultiThreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    List<IgniteFuture<?>> futs0 = new ArrayList<>();
+
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    while (left.getAndDecrement() > 0) {
+                        TreeMap<TestKey, TestValue> map = new TreeMap<>();
+
+                        int keys = 50;
+
+                        for (int k = 0; k < keys; k++)
+                            map.put(new TestKey(rnd.nextInt(10_000)), new TestValue(k));
+
+                        cache.putAll(map);
+
+                        IgniteFuture<?> fut = cache.future();
+
+                        assertNotNull(fut);
+
+                        futs0.add(fut);
+                    }
+
+                    futs.addAll(futs0);
+
+                    return null;
+                }
+            }, 10, "put-thread");
+
+            stopGrid(NODE_CNT);
+
+            assertEquals(ops, futs.size());
+
+            for (IgniteFuture<?> fut : futs)
+                fut.get();
+        }
+    }
+
+    /**
+     * @param threads Number of threads.
+     * @param opsPerThread Number of concurrent async operations per thread.
+     * @throws Exception If failed.
+     */
+    private void putAllAsyncFailover(final int threads, final int opsPerThread) throws Exception {
+        log.info("Start test [threads=" + threads + ", opsPerThread=" + opsPerThread + ']');
+
+        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).<TestKey, TestValue>cache(null).withAsync();
+
+            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("Starting operations [iter=" + iter + ']');
+
+                        List<IgniteFuture<?>> futs = new ArrayList<>(opsPerThread);
+
+                        for (int i = 0; i < opsPerThread; i++) {
+                            TreeMap<TestKey, TestValue> map = new TreeMap<>();
+
+                            int keys = rnd.nextInt(1, 50);
+
+                            for (int k = 0; k < keys; k++)
+                                map.put(new TestKey(rnd.nextInt(10_000)), new TestValue(iter));
+
+                            cache.putAll(map);
+
+                            IgniteFuture<?> fut = cache.future();
+
+                            assertNotNull(fut);
+
+                            futs.add(fut);
+                        }
+
+                        if (time - lastInfo > 5000) {
+                            log.info("Waiting for futures [iter=" + iter + ']');
+
+                            lastInfo = time;
+                        }
+
+                        for (IgniteFuture<?> fut : futs)
+                            fut.get();
+
+                        iter++;
+                    }
+
+                    return null;
+                }
+            }, threads, "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/73ab5e2f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAtomicTest.java
new file mode 100644
index 0000000..6e01a4a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAtomicTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class CacheAsyncOperationsFailoverAtomicTest extends CacheAsyncOperationsFailoverAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/73ab5e2f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
index f3fac23..ba510f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite2.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.*;
 import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
@@ -48,6 +49,9 @@ public class IgniteCacheFailoverTestSuite2 {
 
         suite.addTestSuite(IgniteCacheCrossCacheTxFailoverTest.class);
 
+        suite.addTestSuite(CacheAsyncOperationsFailoverAtomicTest.class);
+        suite.addTestSuite(CacheAsyncOperationsFailoverTxTest.class);
+
         return suite;
     }
 }


[03/50] [abbrv] ignite git commit: IGNITE-1302: Moving some common .Net classes to Ignite.

Posted by ak...@apache.org.
IGNITE-1302: Moving some common .Net classes to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 8df6b935c32d8b801ead4deb0470f84df435ff5a
Parents: 824cfa4
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Aug 26 15:55:24 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 15:55:24 2015 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.csproj                   |  22 ++
 .../Common/AsyncSupportedAttribute.cs           |  33 +++
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |  52 ++++
 .../dotnet/Apache.Ignite.Core/Common/IFuture.cs | 115 ++++++++
 .../Common/IgniteException.cs                   |  66 +++++
 .../Impl/Collections/CollectionExtensions.cs    |  45 +++
 .../Impl/Collections/MultiValueDictionary.cs    | 143 ++++++++++
 .../Impl/Collections/ReadOnlyCollection.cs      | 102 +++++++
 .../Impl/Collections/ReadOnlyDictionary.cs      | 149 ++++++++++
 .../Impl/Common/AsyncResult.cs                  |  71 +++++
 .../Impl/Common/CompletedAsyncResult.cs         |  70 +++++
 .../Common/CopyOnWriteConcurrentDictionary.cs   |  70 +++++
 .../Impl/Common/DelegateConverter.cs            | 253 ++++++++++++++++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    | 286 +++++++++++++++++++
 .../Impl/Common/FutureType.cs                   |  52 ++++
 .../Impl/Common/GridArgumentCheck.cs            |  76 +++++
 .../Impl/Common/IFutureConverter.cs             |  32 +++
 .../Impl/Common/IFutureInternal.cs              |  45 +++
 .../Impl/Common/LoadedAssembliesResolver.cs     |  96 +++++++
 .../Impl/Common/TypeCaster.cs                   |  72 +++++
 20 files changed, 1850 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 658e5fb..12e335a 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -16,6 +16,7 @@
     <PlatformTarget>x64</PlatformTarget>
     <OutputPath>bin\x64\Debug\</OutputPath>
     <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <DefineConstants>DEBUG</DefineConstants>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
     <PlatformTarget>x64</PlatformTarget>
@@ -26,6 +27,7 @@
     <PlatformTarget>x86</PlatformTarget>
     <OutputPath>bin\x86\Debug\</OutputPath>
     <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
+    <DefineConstants>DEBUG</DefineConstants>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
     <PlatformTarget>x86</PlatformTarget>
@@ -47,7 +49,26 @@
     <Reference Include="System.Core" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Common\IgniteException.cs" />
+    <Compile Include="Common\IAsyncSupport.cs" />
+    <Compile Include="Common\IFuture.cs" />
     <Compile Include="Ignition.cs" />
+    <Compile Include="Common\AsyncSupportedAttribute.cs" />
+    <Compile Include="Impl\Collections\CollectionExtensions.cs" />
+    <Compile Include="Impl\Collections\MultiValueDictionary.cs" />
+    <Compile Include="Impl\Collections\ReadOnlyCollection.cs" />
+    <Compile Include="Impl\Collections\ReadOnlyDictionary.cs" />
+    <Compile Include="Impl\Common\AsyncResult.cs" />
+    <Compile Include="Impl\Common\CompletedAsyncResult.cs" />
+    <Compile Include="Impl\Common\CopyOnWriteConcurrentDictionary.cs" />
+    <Compile Include="Impl\Common\DelegateConverter.cs" />
+    <Compile Include="Impl\Common\Future.cs" />
+    <Compile Include="Impl\Common\FutureType.cs" />
+    <Compile Include="Impl\Common\GridArgumentCheck.cs" />
+    <Compile Include="Impl\Common\IFutureConverter.cs" />
+    <Compile Include="Impl\Common\IFutureInternal.cs" />
+    <Compile Include="Impl\Common\LoadedAssembliesResolver.cs" />
+    <Compile Include="Impl\Common\TypeCaster.cs" />
     <Compile Include="Impl\Handle\Handle.cs" />
     <Compile Include="Impl\Handle\HandleRegistry.cs" />
     <Compile Include="Impl\Handle\IHandle.cs" />
@@ -64,6 +85,7 @@
     <Compile Include="Impl\Portable\IO\IPortableStream.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
   </ItemGroup>
+  <ItemGroup />
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
        Other similar extension points exist, see Microsoft.Common.targets.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/AsyncSupportedAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/AsyncSupportedAttribute.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/AsyncSupportedAttribute.cs
new file mode 100644
index 0000000..094a93c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/AsyncSupportedAttribute.cs
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Common
+{
+    using System;
+
+    /// <summary>
+    /// Attribute to indicate that method can be executed asynchronously if async mode is enabled.
+    /// To enable async mode, invoke <see cref="IAsyncSupport{TWithAsync}.WithAsync"/> method on the API.
+    /// The future for the async method can be retrieved via 
+    /// <see cref="IFuture{T}"/> right after the execution of an asynchronous method.
+    /// </summary>
+    [AttributeUsage(AttributeTargets.Method)]
+    public sealed class AsyncSupportedAttribute : Attribute
+    {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
new file mode 100644
index 0000000..f6b6551
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IAsyncSupport.cs
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Common
+{
+    /// <summary>
+    /// Allows to enable asynchronous mode on Grid APIs.
+    /// </summary>
+    /// <typeparam name="TWithAsync">Type of WithAsync method result.</typeparam>
+    public interface IAsyncSupport<out TWithAsync> where TWithAsync : IAsyncSupport<TWithAsync>
+    {
+        /// <summary>
+        /// Gets component with asynchronous mode enabled.
+        /// </summary>
+        /// <returns>Component with asynchronous mode enabled.</returns>
+        TWithAsync WithAsync();
+
+        /// <summary>
+        /// Gets a value indicating whether this instance is in asynchronous mode.
+        /// </summary>
+        /// <value>
+        /// <c>true</c> if asynchronous mode is enabled.
+        /// </value>
+        bool IsAsync { get; }
+
+        /// <summary>
+        /// Gets and resets future for previous asynchronous operation.
+        /// </summary>
+        /// <returns>Future for previous asynchronous operation.</returns>
+        IFuture GetFuture();
+
+        /// <summary>
+        /// Gets and resets future for previous asynchronous operation.
+        /// </summary>
+        /// <returns>Future for previous asynchronous operation.</returns>
+        IFuture<TResult> GetFuture<TResult>();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IFuture.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IFuture.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IFuture.cs
new file mode 100644
index 0000000..2e94cd4
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IFuture.cs
@@ -0,0 +1,115 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Common
+{
+    using System;
+    using System.Threading.Tasks;
+
+    /// <summary>
+    /// Non-generic Future. Represents an asynchronous operation that can return a value.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    public interface IFuture
+    {
+        /// <summary>
+        /// Gets a value indicating whether this instance is done.
+        /// </summary>
+        bool IsDone
+        {
+            get;
+        }
+
+        /// <summary>
+        /// Gets the future result.
+        /// </summary>
+        /// <returns>Future result.</returns>
+        object Get();
+
+        /// <summary>
+        /// Gets the future result with a timeout.
+        /// </summary>
+        /// <param name="timeout">The timeout.</param>
+        /// <returns>
+        /// Future result, if it is obtained within specified timeout; otherwise, throws <see cref="TimeoutException"/>
+        /// </returns>
+        /// <exception cref="TimeoutException">Thrown if Get operation exceeded specified timeout.</exception>
+        object Get(TimeSpan timeout);
+
+        /// <summary>
+        /// Listens this instance and invokes callback upon future completion.
+        /// </summary>
+        /// <param name="callback">The callback to execute upon future completion.</param>
+        void Listen(Action callback);
+
+        /// <summary>
+        /// Listens this instance and invokes callback upon future completion.
+        /// </summary>
+        /// <param name="callback">The callback to execute upon future completion.</param>
+        void Listen(Action<IFuture> callback);
+
+        /// <summary>
+        /// Gets an IAsyncResult indicating the state of this Future.
+        /// </summary>
+        /// <returns>Future state representation in form of IAsyncResult.</returns>
+        IAsyncResult ToAsyncResult();
+
+        /// <summary>
+        /// Gets a Task that returns the result of this Future.
+        /// </summary>
+        /// <returns>Task that completes when this future gets done and returns the result.</returns>
+        Task<object> ToTask();
+    }
+
+    /// <summary>
+    /// Generic Future. Represents an asynchronous operation that can return a value.
+    /// <para/>
+    /// All members are thread-safe and may be used concurrently from multiple threads.
+    /// </summary>
+    /// <typeparam name="T">Future result type.</typeparam>
+    public interface IFuture<T> : IFuture
+    {
+        /// <summary>
+        /// Gets the future result.
+        /// </summary>
+        /// <returns>Future result.</returns>
+        new T Get();
+
+        /// <summary>
+        /// Gets the future result with a timeout.
+        /// </summary>
+        /// <param name="timeout">The timeout.</param>
+        /// <returns>
+        /// Future result, if it is obtained within specified timeout; otherwise, throws <see cref="TimeoutException"/>
+        /// </returns>
+        /// <exception cref="TimeoutException">Thrown if Get operation exceeded specified timeout.</exception>
+        new T Get(TimeSpan timeout);
+
+        /// <summary>
+        /// Gets a Task that returns the result of this Future.
+        /// </summary>
+        /// <returns>Task that completes when this future gets done and returns the result.</returns>
+        new Task<T> ToTask();
+
+        /// <summary>
+        /// Listens this instance and invokes callback upon future completion.
+        /// </summary>
+        /// <param name="callback">The callback to execute upon future completion.</param>
+        void Listen(Action<IFuture<T>> callback);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
new file mode 100644
index 0000000..4626407
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Common/IgniteException.cs
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Common
+{
+    using System;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// General grid exception. Indicates any error condition within Grid.
+    /// </summary>
+    [Serializable]
+    public class IgniteException : Exception
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteException"/> class.
+        /// </summary>
+        public IgniteException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteException" /> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public IgniteException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteException" /> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public IgniteException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected IgniteException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/CollectionExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/CollectionExtensions.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/CollectionExtensions.cs
new file mode 100644
index 0000000..57295cb
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/CollectionExtensions.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Collections
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Collection extension methods.
+    /// </summary>
+    public static class CollectionExtensions
+    {
+        /// <summary>
+        /// Returns a read-only System.Collections.Generic.IDictionary{K, V} wrapper for the current collection.
+        /// </summary>
+        public static IDictionary<TKey, TValue> AsReadOnly<TKey, TValue>(this IDictionary<TKey, TValue> dict)
+        {
+            return new ReadOnlyDictionary<TKey, TValue>(dict);
+        }
+
+        /// <summary>
+        /// Returns a read-only System.Collections.Generic.ICollection{K, V} wrapper for the current collection.
+        /// </summary>
+        public static ICollection<T> AsReadOnly<T>(this ICollection<T> col)
+        {
+            var list = col as List<T>;
+
+            return list != null ? (ICollection<T>) list.AsReadOnly() : new ReadOnlyCollection<T>(col);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
new file mode 100644
index 0000000..bd7e895
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
@@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Collections
+{
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Multiple-values-per-key dictionary.
+    /// </summary>
+    public class MultiValueDictionary<TKey, TValue>
+    {
+        /** Inner dictionary */
+        private readonly Dictionary<TKey, object> _dict = new Dictionary<TKey, object>();
+
+        /// <summary>
+        /// Adds a value.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="val">The value.</param>
+        public void Add(TKey key, TValue val)
+        {
+            object val0;
+
+            if (_dict.TryGetValue(key, out val0))
+            {
+                var list = val0 as List<TValue>;
+
+                if (list != null)
+                    list.Add(val);
+                else
+                    _dict[key] = new List<TValue> {(TValue) val0, val};
+            }
+            else
+                _dict[key] = val;
+        }
+
+        /// <summary>
+        /// Tries the get a value. In case of multiple values for a key, returns the last one.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="val">The value.</param>
+        /// <returns>True if value has been found for specified key; otherwise false.</returns>
+        public bool TryGetValue(TKey key, out TValue val)
+        {
+            object val0;
+            
+            if (!_dict.TryGetValue(key, out val0))
+            {
+                val = default(TValue);
+                return false;
+            }
+
+            var list = val0 as List<TValue>;
+
+            if (list != null)
+                val = list[list.Count - 1];
+            else
+                val = (TValue) val0;
+
+            return true;
+        }
+
+        /// <summary>
+        /// Removes the specified value for the specified key.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="val">The value.</param>
+        public void Remove(TKey key, TValue val)
+        {
+            object val0;
+
+            if (!_dict.TryGetValue(key, out val0))
+                return;
+
+            var list = val0 as List<TValue>;
+
+            if (list != null)
+            {
+                list.Remove(val);
+
+                if (list.Count == 0)
+                    _dict.Remove(key);
+            }
+            else if (Equals(val0, val))
+                _dict.Remove(key);
+        }
+
+        /// <summary>
+        /// Removes the last value for the specified key and returns it.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="val">The value.</param>
+        /// <returns>True if value has been found for specified key; otherwise false.</returns>
+        public bool TryRemove(TKey key, out TValue val)
+        {
+            object val0;
+
+            if (!_dict.TryGetValue(key, out val0))
+            {
+                val = default(TValue);
+
+                return false;
+            }
+
+            var list = val0 as List<TValue>;
+
+            if (list != null)
+            {
+                var index = list.Count - 1;
+
+                val = list[index];
+
+                list.RemoveAt(index);
+
+                if (list.Count == 0)
+                    _dict.Remove(key);
+
+                return true;
+            }
+            
+            val = (TValue) val0;
+
+            _dict.Remove(key);
+
+            return true;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyCollection.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyCollection.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyCollection.cs
new file mode 100644
index 0000000..23cae6b
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyCollection.cs
@@ -0,0 +1,102 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Collections
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Read-only wrapper over ICollection{T}.
+    /// </summary>
+    internal struct ReadOnlyCollection<T> : ICollection<T>
+    {
+        /** Wrapped collection. */
+        private readonly ICollection<T> _col;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ReadOnlyCollection{T}"/> class.
+        /// </summary>
+        public ReadOnlyCollection(ICollection<T> col)
+        {
+            _col = col;
+        }
+
+        /** <inheritdoc /> */
+        public IEnumerator<T> GetEnumerator()
+        {
+            return _col.GetEnumerator();
+        }
+
+        /** <inheritdoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return ((IEnumerable) _col).GetEnumerator();
+        }
+
+        /** <inheritdoc /> */
+        public void Add(T item)
+        {
+            throw GetReadOnlyException();
+        }
+
+        /** <inheritdoc /> */
+        public void Clear()
+        {
+            throw GetReadOnlyException();
+        }
+
+        /** <inheritdoc /> */
+        public bool Contains(T item)
+        {
+            return _col.Contains(item);
+        }
+
+        /** <inheritdoc /> */
+        public void CopyTo(T[] array, int arrayIndex)
+        {
+            _col.CopyTo(array, arrayIndex);
+        }
+
+        /** <inheritdoc /> */
+        public bool Remove(T item)
+        {
+            throw GetReadOnlyException();
+        }
+
+        /** <inheritdoc /> */
+        public int Count
+        {
+            get { return _col.Count; }
+        }
+
+        /** <inheritdoc /> */
+        public bool IsReadOnly
+        {
+            get { return true; }
+        }
+
+        /// <summary>
+        /// Gets the readonly exception.
+        /// </summary>
+        private static Exception GetReadOnlyException()
+        {
+            return new NotSupportedException("Collection is read-only.");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
new file mode 100644
index 0000000..60ec9d0
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
@@ -0,0 +1,149 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Collections
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+
+    /// <summary>
+    /// Read-only wrapper over IDictionary{K, V}.
+    /// </summary>
+    internal struct ReadOnlyDictionary<TKey, TValue> : IDictionary<TKey, TValue>
+    {
+        /** Inner dict. */
+        private readonly IDictionary<TKey, TValue> _dict;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ReadOnlyDictionary{K, V}"/> class.
+        /// </summary>
+        /// <param name="dict">The dictionary to wrap.</param>
+        public ReadOnlyDictionary(IDictionary<TKey, TValue> dict)
+        {
+            Debug.Assert(dict != null);
+
+            _dict = dict;
+        }
+
+        /** <inheritdoc /> */
+        public IEnumerator<KeyValuePair<TKey, TValue>> GetEnumerator()
+        {
+            return _dict.GetEnumerator();
+        }
+
+        /** <inheritdoc /> */
+        IEnumerator IEnumerable.GetEnumerator()
+        {
+            return ((IEnumerable) _dict).GetEnumerator();
+        }
+
+        /** <inheritdoc /> */
+        public void Add(KeyValuePair<TKey, TValue> item)
+        {
+            throw GetReadonlyException();
+        }
+
+        /** <inheritdoc /> */
+        public void Clear()
+        {
+            throw GetReadonlyException();
+        }
+
+        /** <inheritdoc /> */
+        public bool Contains(KeyValuePair<TKey, TValue> item)
+        {
+            return _dict.Contains(item);
+        }
+
+        /** <inheritdoc /> */
+        public void CopyTo(KeyValuePair<TKey, TValue>[] array, int arrayIndex)
+        {
+            _dict.CopyTo(array, arrayIndex);
+        }
+
+        /** <inheritdoc /> */
+        public bool Remove(KeyValuePair<TKey, TValue> item)
+        {
+            throw GetReadonlyException();
+        }
+
+        /** <inheritdoc /> */
+        public int Count
+        {
+            get { return _dict.Count; }
+        }
+
+        /** <inheritdoc /> */
+        public bool IsReadOnly
+        {
+            get { return true; }
+        }
+
+        /** <inheritdoc /> */
+        public bool ContainsKey(TKey key)
+        {
+            return _dict.ContainsKey(key);
+        }
+
+        /** <inheritdoc /> */
+        public void Add(TKey key, TValue value)
+        {
+            throw GetReadonlyException();
+        }
+
+        /** <inheritdoc /> */
+        public bool Remove(TKey key)
+        {
+            return _dict.Remove(key);
+        }
+
+        /** <inheritdoc /> */
+        public bool TryGetValue(TKey key, out TValue value)
+        {
+            return _dict.TryGetValue(key, out value);
+        }
+
+        /** <inheritdoc /> */
+        public TValue this[TKey key]
+        {
+            get { return _dict[key]; }
+            set { throw GetReadonlyException(); }
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<TKey> Keys
+        {
+            get { return _dict.Keys; }
+        }
+
+        /** <inheritdoc /> */
+        public ICollection<TValue> Values
+        {
+            get { return _dict.Values; }
+        }
+
+        /// <summary>
+        /// Gets the readonly exception.
+        /// </summary>
+        private static Exception GetReadonlyException()
+        {
+            return new NotSupportedException("Dictionary is read-only.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/AsyncResult.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/AsyncResult.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/AsyncResult.cs
new file mode 100644
index 0000000..4e5c396
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/AsyncResult.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Adapts IGridFuture to the IAsyncResult.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable",
+        Justification = "Implementing IDisposable has no point since we return this class as IAsyncResult " +
+                        "to the client, and IAsyncResult is not IDisposable.")]
+    public class AsyncResult : IAsyncResult
+    {
+        /** */
+        private readonly ManualResetEvent _waitHandle;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="AsyncResult"/> class.
+        /// </summary>
+        /// <param name="fut">The future to wrap.</param>
+        public AsyncResult(IFuture fut)
+        {
+            _waitHandle = new ManualResetEvent(false);
+
+            fut.Listen(() => _waitHandle.Set());
+        }
+
+        /** <inheritdoc /> */
+        public bool IsCompleted
+        {
+            get { return _waitHandle.WaitOne(0); }
+        }
+
+        /** <inheritdoc /> */
+        public WaitHandle AsyncWaitHandle
+        {
+            get { return _waitHandle; }
+        }
+
+        /** <inheritdoc /> */
+        public object AsyncState
+        {
+            get { return null; }
+        }
+
+        /** <inheritdoc /> */
+        public bool CompletedSynchronously
+        {
+            get { return false; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CompletedAsyncResult.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CompletedAsyncResult.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CompletedAsyncResult.cs
new file mode 100644
index 0000000..14195fd
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CompletedAsyncResult.cs
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+
+    /// <summary>
+    /// Represents an IAsyncResult that is completed.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable", 
+        Justification = "Implementing IDisposable has no point since we return this class as IAsyncResult " +
+                        "to the client, and IAsyncResult is not IDisposable.")]
+    public class CompletedAsyncResult : IAsyncResult
+    {
+        /** Singleton instance. */
+        public static readonly IAsyncResult Instance = new CompletedAsyncResult();
+
+        /** */
+        private readonly WaitHandle _asyncWaitHandle = new ManualResetEvent(true);
+
+        /// <summary>
+        /// Prevents a default instance of the <see cref="CompletedAsyncResult"/> class from being created.
+        /// </summary>
+        private CompletedAsyncResult()
+        {
+            // No-op.
+        }
+
+        /** <inheritdoc /> */
+        public bool IsCompleted
+        {
+            get { return true; }
+        }
+
+        /** <inheritdoc /> */
+        public WaitHandle AsyncWaitHandle
+        {
+            get { return _asyncWaitHandle; }
+        }
+
+        /** <inheritdoc /> */
+        public object AsyncState
+        {
+            get { return null; }
+        }
+
+        /** <inheritdoc /> */
+        public bool CompletedSynchronously
+        {
+            get { return false; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
new file mode 100644
index 0000000..fa785b2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/CopyOnWriteConcurrentDictionary.cs
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Concurrent dictionary with CopyOnWrite mechanism inside. 
+    /// Good for frequent reads / infrequent writes scenarios.
+    /// </summary>
+    public class CopyOnWriteConcurrentDictionary<TKey, TValue>
+    {
+        /** */
+        private volatile Dictionary<TKey, TValue> _dict = new Dictionary<TKey, TValue>();
+
+        /// <summary>
+        /// Gets the value associated with the specified key.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="val">The value.</param>
+        /// <returns>true if the dictionary contains an element with the specified key; otherwise, false.</returns>
+        public bool TryGetValue(TKey key, out TValue val)
+        {
+            return _dict.TryGetValue(key, out val);
+        }
+
+        /// <summary>
+        /// Adds a key/value pair if the key does not already exist.
+        /// </summary>
+        /// <param name="key">The key.</param>
+        /// <param name="valueFactory">The function used to generate a value for the key.</param>
+        /// <returns>The value for the key.</returns>
+        public TValue GetOrAdd(TKey key, Func<TKey, TValue> valueFactory)
+        {
+            lock (this)
+            {
+                TValue res;
+
+                if (_dict.TryGetValue(key, out res))
+                    return res;
+
+                var dict0 = new Dictionary<TKey, TValue>(_dict);
+
+                res = valueFactory(key);
+
+                dict0[key] = res;
+
+                _dict = dict0;
+
+                return res;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
new file mode 100644
index 0000000..7f83588
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/DelegateConverter.cs
@@ -0,0 +1,253 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Diagnostics;
+    using System.Linq.Expressions;
+    using System.Reflection;
+    using System.Reflection.Emit;
+
+    /// <summary>
+    /// Converts generic and non-generic delegates.
+    /// </summary>
+    public static class DelegateConverter
+    {
+        /** */
+        private const string DefaultMethodName = "Invoke";
+        
+        /// <summary>
+        /// Compiles a function without arguments.
+        /// </summary>
+        /// <param name="targetType">Type of the target.</param>
+        /// <returns>Compiled function that calls specified method on specified target.</returns>
+        public static Func<object, object> CompileFunc(Type targetType)
+        {
+            var method = targetType.GetMethod(DefaultMethodName);
+
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, targetType);
+
+            var callExpr = Expression.Call(targetParamConverted, method);
+            var convertResultExpr = Expression.Convert(callExpr, typeof(object));
+
+            return Expression.Lambda<Func<object, object>>(convertResultExpr, targetParam).Compile();
+        }
+
+        /// <summary>
+        /// Compiles a function with arbitrary number of arguments.
+        /// </summary>
+        /// <typeparam name="T">Resulting delegate type.</typeparam>
+        /// <param name="targetType">Type of the target.</param>
+        /// <param name="argTypes">Argument types.</param>
+        /// <param name="convertToObject">
+        /// Flags that indicate whether func params and/or return value should be converted from/to object.
+        /// </param>
+        /// <param name="methodName">Name of the method.</param>
+        /// <returns>
+        /// Compiled function that calls specified method on specified target.
+        /// </returns>
+        public static T CompileFunc<T>(Type targetType, Type[] argTypes, bool[] convertToObject = null,
+            string methodName = null)
+            where T : class
+        {
+            var method = targetType.GetMethod(methodName ?? DefaultMethodName, argTypes);
+
+            return CompileFunc<T>(targetType, method, argTypes, convertToObject);
+        }
+
+        /// <summary>
+        /// Compiles a function with arbitrary number of arguments.
+        /// </summary>
+        /// <typeparam name="T">Resulting delegate type.</typeparam>
+        /// <param name="method">Method.</param>
+        /// <param name="targetType">Type of the target.</param>
+        /// <param name="argTypes">Argument types.</param>
+        /// <param name="convertToObject">
+        /// Flags that indicate whether func params and/or return value should be converted from/to object.
+        /// </param>
+        /// <returns>
+        /// Compiled function that calls specified method on specified target.
+        /// </returns>
+        public static T CompileFunc<T>(Type targetType, MethodInfo method, Type[] argTypes, 
+            bool[] convertToObject = null)
+            where T : class
+        {
+            if (argTypes == null)
+            {
+                var args = method.GetParameters();
+                argTypes = new Type[args.Length];
+
+                for (int i = 0; i < args.Length; i++)
+                    argTypes[i] = args[i].ParameterType;
+            }
+
+            Debug.Assert(convertToObject == null || (convertToObject.Length == argTypes.Length + 1));
+            Debug.Assert(method != null);
+
+            targetType = method.IsStatic ? null : (targetType ?? method.DeclaringType);
+
+            var targetParam = Expression.Parameter(typeof(object));
+            
+            Expression targetParamConverted = null;
+            ParameterExpression[] argParams;
+            int argParamsOffset = 0;
+
+            if (targetType != null)
+            {
+                targetParamConverted = Expression.Convert(targetParam, targetType);
+                argParams = new ParameterExpression[argTypes.Length + 1];
+                argParams[0] = targetParam;
+                argParamsOffset = 1;
+            }
+            else
+                argParams = new ParameterExpression[argTypes.Length];  // static method
+
+            var argParamsConverted = new Expression[argTypes.Length];
+
+            for (var i = 0; i < argTypes.Length; i++)
+            {
+                if (convertToObject == null || convertToObject[i])
+                {
+                    var argParam = Expression.Parameter(typeof (object));
+                    argParams[i + argParamsOffset] = argParam;
+                    argParamsConverted[i] = Expression.Convert(argParam, argTypes[i]);
+                }
+                else
+                {
+                    var argParam = Expression.Parameter(argTypes[i]);
+                    argParams[i + argParamsOffset] = argParam;
+                    argParamsConverted[i] = argParam;
+                }
+            }
+
+            Expression callExpr = Expression.Call(targetParamConverted, method, argParamsConverted);
+
+            if (convertToObject == null || convertToObject[argTypes.Length])
+                callExpr = Expression.Convert(callExpr, typeof(object));
+
+            return Expression.Lambda<T>(callExpr, argParams).Compile();
+        }
+
+        /// <summary>
+        /// Compiles a generic ctor with arbitrary number of arguments.
+        /// </summary>
+        /// <typeparam name="T">Result func type.</typeparam>
+        /// <param name="type">Type to be created by ctor.</param>
+        /// <param name="argTypes">Argument types.</param>
+        /// <param name="convertResultToObject">if set to <c>true</c> [convert result to object].
+        /// Flag that indicates whether ctor return value should be converted to object.
+        /// </param>
+        /// <returns>
+        /// Compiled generic constructor.
+        /// </returns>
+        public static T CompileCtor<T>(Type type, Type[] argTypes, bool convertResultToObject = true)
+        {
+            var ctor = type.GetConstructor(argTypes);
+
+            Debug.Assert(ctor != null);
+
+            var args = new ParameterExpression[argTypes.Length];
+            var argsConverted = new Expression[argTypes.Length];
+
+            for (var i = 0; i < argTypes.Length; i++)
+            {
+                var arg = Expression.Parameter(typeof(object));
+                args[i] = arg;
+                argsConverted[i] = Expression.Convert(arg, argTypes[i]);
+            }
+
+            Expression ctorExpr = Expression.New(ctor, argsConverted);  // ctor takes args of specific types
+
+            if (convertResultToObject)
+                ctorExpr = Expression.Convert(ctorExpr, typeof (object)); // convert ctor result to object
+
+            return Expression.Lambda<T>(ctorExpr, args).Compile();  // lambda takes args as objects
+        }
+
+        /// <summary>
+        /// Compiles the field setter.
+        /// </summary>
+        /// <param name="field">The field.</param>
+        /// <returns>Compiled field setter.</returns>
+        public static Action<object, object> CompileFieldSetter(FieldInfo field)
+        {
+            Debug.Assert(field != null);
+            Debug.Assert(field.DeclaringType != null);   // non-static
+
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, field.DeclaringType);
+
+            var valParam = Expression.Parameter(typeof(object));
+            var valParamConverted = Expression.Convert(valParam, field.FieldType);
+
+            var assignExpr = Expression.Call(GetWriteFieldMethod(field), targetParamConverted, valParamConverted);
+
+            return Expression.Lambda<Action<object, object>>(assignExpr, targetParam, valParam).Compile();
+        }
+
+        /// <summary>
+        /// Compiles the property setter.
+        /// </summary>
+        /// <param name="prop">The property.</param>
+        /// <returns>Compiled property setter.</returns>
+        public static Action<object, object> CompilePropertySetter(PropertyInfo prop)
+        {
+            Debug.Assert(prop != null);
+            Debug.Assert(prop.DeclaringType != null);   // non-static
+
+            var targetParam = Expression.Parameter(typeof(object));
+            var targetParamConverted = Expression.Convert(targetParam, prop.DeclaringType);
+
+            var valParam = Expression.Parameter(typeof(object));
+            var valParamConverted = Expression.Convert(valParam, prop.PropertyType);
+
+            var fld = Expression.Property(targetParamConverted, prop);
+
+            var assignExpr = Expression.Assign(fld, valParamConverted);
+
+            return Expression.Lambda<Action<object, object>>(assignExpr, targetParam, valParam).Compile();
+        }
+
+        /// <summary>
+        /// Gets a method to write a field (including private and readonly).
+        /// NOTE: Expression Trees can't write readonly fields.
+        /// </summary>
+        /// <param name="field">The field.</param>
+        /// <returns>Resulting MethodInfo.</returns>
+        public static DynamicMethod GetWriteFieldMethod(FieldInfo field)
+        {
+            Debug.Assert(field != null);
+
+            var module = Assembly.GetExecutingAssembly().GetModules()[0];
+
+            var method = new DynamicMethod(string.Empty, null, new[] { field.DeclaringType, field.FieldType }, module,
+                true);
+
+            var il = method.GetILGenerator();
+
+            il.Emit(OpCodes.Ldarg_0);
+            il.Emit(OpCodes.Ldarg_1);
+            il.Emit(OpCodes.Stfld, field);
+            il.Emit(OpCodes.Ret);
+
+            return method;
+        }
+
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
new file mode 100644
index 0000000..c62cfd2
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -0,0 +1,286 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Threading;
+    using System.Threading.Tasks;
+    
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Grid future implementation.
+    /// </summary>
+    [SuppressMessage("ReSharper", "ParameterHidesMember")]
+    public sealed class Future<T> : IFutureInternal, IFuture<T>
+    {
+        /** Converter. */
+        private readonly IFutureConverter<T> _converter;
+
+        /** Result. */
+        private T _res;
+
+        /** Caught cxception. */
+        private Exception _err;
+
+        /** Done flag. */
+        private volatile bool _done;
+
+        /** Listener(s). Either Action or List{Action}. */
+        private object _callbacks;
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="converter">Future result marshaller and converter.</param>
+        public Future(IFutureConverter<T> converter = null)
+        {
+            _converter = converter;
+        }
+
+        /** <inheritdoc/> */
+        public bool IsDone
+        {
+            get { return _done; }
+        }
+
+        /** <inheritdoc/> */
+        public T Get()
+        {
+            if (!_done)
+            {
+                lock (this)
+                {
+                    while (!_done)
+                        Monitor.Wait(this);
+                }
+            }
+
+            return Get0();
+        }
+
+        /** <inheritdoc/> */
+        public T Get(TimeSpan timeout)
+        {
+            long ticks = timeout.Ticks;
+
+            if (ticks < 0)
+                throw new ArgumentException("Timeout cannot be negative.");
+
+            if (ticks == 0)
+                return Get();
+
+            if (!_done)
+            {
+                // Fallback to locked mode.
+                lock (this)
+                {
+                    long endTime = DateTime.Now.Ticks + ticks;
+
+                    if (!_done)
+                    {
+                        while (true)
+                        {
+                            Monitor.Wait(this, timeout);
+
+                            if (_done)
+                                break;
+
+                            ticks = endTime - DateTime.Now.Ticks;
+
+                            if (ticks <= 0)
+                                throw new TimeoutException("Timeout waiting for future completion.");
+
+                            timeout = TimeSpan.FromTicks(ticks);
+                        }
+                    }
+                }
+            }
+
+            return Get0();
+        }
+
+        /** <inheritdoc/> */
+        public void Listen(Action callback)
+        {
+            Listen((Action<IFuture<T>>) (fut => callback()));
+        }
+
+        /** <inheritdoc/> */
+        public void Listen(Action<IFuture> callback)
+        {
+            Listen((Action<IFuture<T>>)callback);
+        }
+
+        /** <inheritdoc/> */
+        public void Listen(Action<IFuture<T>> callback)
+        {
+            GridArgumentCheck.NotNull(callback, "callback");
+
+            if (!_done)
+            {
+                lock (this)
+                {
+                    if (!_done)
+                    {
+                        AddCallback(callback);
+
+                        return;
+                    }
+                }
+            }
+
+            callback(this);
+        }
+
+        /// <summary>
+        /// Get result or throw an error.
+        /// </summary>
+        private T Get0()
+        {
+            if (_err != null)
+                throw _err;
+
+            return _res;
+        }
+
+        /** <inheritdoc/> */
+        public IAsyncResult ToAsyncResult()
+        {
+            return _done ? CompletedAsyncResult.Instance : new AsyncResult(this);
+        }
+
+        /** <inheritdoc/> */
+        Task<object> IFuture.ToTask()
+        {
+            return Task.Factory.FromAsync(ToAsyncResult(), x => (object) Get());
+        }
+
+        /** <inheritdoc/> */
+        public Task<T> ToTask()
+        {
+            return Task.Factory.FromAsync(ToAsyncResult(), x => Get());
+        }
+
+        /** <inheritdoc/> */
+        object IFuture.Get(TimeSpan timeout)
+        {
+            return Get(timeout);
+        }
+
+        /** <inheritdoc/> */
+        object IFuture.Get()
+        {
+            return Get();
+        }
+
+        /** <inheritdoc /> */
+        public void OnResult(IPortableStream stream)
+        {
+            try
+            {
+                OnResult(_converter.Convert(stream));
+            }
+            catch (Exception ex)
+            {
+                OnError(ex);
+            }
+        }
+
+        /** <inheritdoc /> */
+        public void OnError(Exception err)
+        {
+            OnDone(default(T), err);
+        }
+
+        /** <inheritdoc /> */
+        public void OnNullResult()
+        {
+            OnResult(default(T));
+        }
+
+        /// <summary>
+        /// Set result.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        internal void OnResult(T res)
+        {
+            OnDone(res, null);
+        }
+
+        /// <summary>
+        /// Set future to Done state.
+        /// </summary>
+        /// <param name="res">Result.</param>
+        /// <param name="err">Error.</param>
+        public void OnDone(T res, Exception err)
+        {
+            object callbacks0 = null;
+
+            lock (this)
+            {
+                if (!_done)
+                {
+                    _res = res;
+                    _err = err;
+
+                    _done = true;
+
+                    Monitor.PulseAll(this);
+
+                    // Notify listeners outside the lock
+                    callbacks0 = _callbacks;
+                    _callbacks = null;
+                }
+            }
+
+            if (callbacks0 != null)
+            {
+                var list = callbacks0 as List<Action<IFuture<T>>>;
+
+                if (list != null)
+                    list.ForEach(x => x(this));
+                else
+                    ((Action<IFuture<T>>) callbacks0)(this);
+            }
+        }
+
+        /// <summary>
+        /// Adds a callback.
+        /// </summary>
+        private void AddCallback(Action<IFuture<T>> callback)
+        {
+            if (_callbacks == null)
+            {
+                _callbacks = callback;
+
+                return;
+            }
+
+            var list = _callbacks as List<Action<IFuture<T>>> ??
+                new List<Action<IFuture<T>>> {(Action<IFuture<T>>) _callbacks};
+
+            list.Add(callback);
+
+            _callbacks = list;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
new file mode 100644
index 0000000..0beff04
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/FutureType.cs
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    /// <summary>
+    /// Future types.
+    /// </summary>
+    public enum FutureType
+    {
+        /** Future type: byte. */
+        Byte = 1,
+
+        /** Future type: boolean. */
+        Bool = 2,
+
+        /** Future type: short. */
+        Short = 3,
+
+        /** Future type: char. */
+        Char = 4,
+
+        /** Future type: int. */
+        Int = 5,
+
+        /** Future type: float. */
+        Float = 6,
+
+        /** Future type: long. */
+        Long = 7,
+
+        /** Future type: double. */
+        Double = 8,
+
+        /** Future type: object. */
+        Object = 9
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs
new file mode 100644
index 0000000..a1fadfe
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/GridArgumentCheck.cs
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Collections.Generic;
+
+    /// <summary>
+    /// Arguments check helpers.
+    /// </summary>
+    public static class GridArgumentCheck
+    {
+        /// <summary>
+        /// Throws an ArgumentNullException if specified arg is null.
+        /// </summary>
+        /// <param name="arg">The argument.</param>
+        /// <param name="argName">Name of the argument.</param>
+        public static void NotNull(object arg, string argName)
+        {
+            if (arg == null)
+                throw new ArgumentNullException(argName);
+        }
+
+        /// <summary>
+        /// Throws an ArgumentException if specified arg is null or empty string.
+        /// </summary>
+        /// <param name="arg">The argument.</param>
+        /// <param name="argName">Name of the argument.</param>
+        public static void NotNullOrEmpty(string arg, string argName)
+        {
+            if (string.IsNullOrEmpty(arg))
+                throw new ArgumentException(string.Format("'{0}' argument should not be null or empty.", argName),
+                    argName);
+        }
+
+        /// <summary>
+        /// Throws an ArgumentException if specified arg is null or empty string.
+        /// </summary>
+        /// <param name="collection">The collection.</param>
+        /// <param name="argName">Name of the argument.</param>
+        public static void NotNullOrEmpty<T>(ICollection<T> collection, string argName)
+        {
+            if (collection == null || collection.Count == 0)
+                throw new ArgumentException(string.Format("'{0}' argument should not be null or empty.", argName),
+                    argName);
+        }
+
+        /// <summary>
+        /// Throws an ArgumentException if specified condition is false.
+        /// </summary>
+        /// <param name="condition">Condition.</param>
+        /// <param name="argName">Name of the argument.</param>
+        /// <param name="message">Message.</param>
+        public static void Ensure(bool condition, string argName, string message)
+        {
+            if (!condition)
+                throw new ArgumentException(string.Format("'{0}' argument is invalid: {1}", argName, message), 
+                    argName);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
new file mode 100644
index 0000000..4169c61
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureConverter.cs
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Marshals and converts future value.
+    /// </summary>
+    public interface IFutureConverter<out T>
+    {
+        /// <summary>
+        /// Reads and converts a value.
+        /// </summary>
+        T Convert(IPortableStream stream);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
new file mode 100644
index 0000000..8547545
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/IFutureInternal.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Portable.IO;
+
+    /// <summary>
+    /// Internal future interface.
+    /// </summary>
+    public interface IFutureInternal
+    {
+        /// <summary>
+        /// Set result from stream.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        void OnResult(IPortableStream stream);
+
+        /// <summary>
+        /// Set null result.
+        /// </summary>
+        void OnNullResult();
+
+        /// <summary>
+        /// Set error result.
+        /// </summary>
+        /// <param name="err">Exception.</param>
+        void OnError(Exception err);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
new file mode 100644
index 0000000..c158d5c
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/LoadedAssembliesResolver.cs
@@ -0,0 +1,96 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Reflection;
+
+    /// <summary>
+    /// Resolves loaded assemblies by name.
+    /// </summary>
+    public class LoadedAssembliesResolver
+    {
+        // The lazy singleton instance.
+        private static readonly Lazy<LoadedAssembliesResolver> LazyInstance = new Lazy<LoadedAssembliesResolver>();
+
+        // Assemblies map.
+        private volatile Dictionary<string, Assembly> _map;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="LoadedAssembliesResolver"/> class.
+        /// </summary>
+        public LoadedAssembliesResolver()
+        {
+            lock (this)
+            {
+                AppDomain.CurrentDomain.AssemblyLoad += CurrentDomain_AssemblyLoad;
+
+                UpdateMap();
+            }
+        }
+
+        /// <summary>
+        /// Handles the AssemblyLoad event of the AppDomain.
+        /// </summary>
+        /// <param name="sender">The source of the event.</param>
+        /// <param name="args">The <see cref="AssemblyLoadEventArgs"/> instance containing the event data.</param>
+        private void CurrentDomain_AssemblyLoad(object sender, AssemblyLoadEventArgs args)
+        {
+            lock (this)
+            {
+                UpdateMap();
+            }
+        }
+
+        /// <summary>
+        /// Updates the assembly map according to the current list of loaded assemblies.
+        /// </summary>
+        private void UpdateMap()
+        {
+            var assemblies = AppDomain.CurrentDomain.GetAssemblies();
+
+            _map = new Dictionary<string, Assembly>(assemblies.Length);
+
+            foreach (var assembly in assemblies)
+                _map[assembly.FullName] = assembly;
+        }
+
+        /// <summary>
+        /// Gets the singleton instance.
+        /// </summary>
+        public static LoadedAssembliesResolver Instance
+        {
+            get { return LazyInstance.Value; }
+        }
+
+        /// <summary>
+        /// Gets the assembly by name.
+        /// </summary>
+        /// <param name="assemblyName">Name of the assembly.</param>
+        /// <returns>Assembly with specified name, or null.</returns>
+        [SuppressMessage("ReSharper", "InconsistentlySynchronizedField")]
+        public Assembly GetAssembly(string assemblyName)
+        {
+            Assembly asm;
+
+            return _map.TryGetValue(assemblyName, out asm) ? asm : null;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8df6b935/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/TypeCaster.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/TypeCaster.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/TypeCaster.cs
new file mode 100644
index 0000000..d0dd2a9
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Impl/Common/TypeCaster.cs
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Linq.Expressions;
+
+    /// <summary>
+    /// Does type casts without extra boxing. 
+    /// Should be used when casting compile-time incompatible value types instead of "(T)(object)x".
+    /// </summary>
+    /// <typeparam name="T">Target type</typeparam>
+    public static class TypeCaster<T>
+    {
+        /// <summary>
+        /// Efficiently casts an object from TFrom to T.
+        /// Does not cause boxing for value types.
+        /// </summary>
+        /// <typeparam name="TFrom">Source type to cast from.</typeparam>
+        /// <param name="obj">The object to cast.</param>
+        /// <returns>Casted object.</returns>
+        public static T Cast<TFrom>(TFrom obj)
+        {
+            return Casters<TFrom>.Caster(obj);
+        }
+
+        /// <summary>
+        /// Inner class serving as a cache.
+        /// </summary>
+        private static class Casters<TFrom>
+        {
+            /// <summary>
+            /// Compiled caster delegate.
+            /// </summary>
+            internal static readonly Func<TFrom, T> Caster = Compile();
+
+            /// <summary>
+            /// Compiles caster delegate.
+            /// </summary>
+            private static Func<TFrom, T> Compile()
+            {
+                if (typeof (T) == typeof (TFrom))
+                {
+                    // Just return what we have
+                    var pExpr = Expression.Parameter(typeof(TFrom));
+                    
+                    return Expression.Lambda<Func<TFrom, T>>(pExpr, pExpr).Compile();
+                }
+
+                var paramExpr = Expression.Parameter(typeof(TFrom));
+                var convertExpr = Expression.Convert(paramExpr, typeof(T));
+
+                return Expression.Lambda<Func<TFrom, T>>(convertExpr, paramExpr).Compile();
+            }
+        }
+    }
+}
\ No newline at end of file


[36/50] [abbrv] ignite git commit: Fixing hang for concurrent cache start

Posted by ak...@apache.org.
Fixing hang for concurrent cache start


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

Branch: refs/heads/ignite-843
Commit: 56fd27251f3f66ad80db9a1719b722051cdff3ab
Parents: ea75a5a
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Aug 27 18:51:45 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Aug 27 18:51:45 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java        | 10 +++++++++-
 .../processors/cache/GridCacheSharedContext.java    | 16 +++++++++-------
 .../near/GridNearOptimisticTxPrepareFuture.java     |  3 +++
 .../cache/transactions/IgniteTxAdapter.java         |  1 +
 4 files changed, 22 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/56fd2725/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index c128aa6..a7fb460 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -516,7 +517,14 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     private IgniteInternalFuture<Object> startFuture(GridCacheMessage cacheMsg) {
         int cacheId = cacheMsg.cacheId();
 
-        return cacheId != 0 ? cctx.cacheContext(cacheId).preloader().startFuture() : cctx.preloadersStartFuture();
+        if (cacheId != 0)
+            return cctx.cacheContext(cacheId).preloader().startFuture();
+        else {
+            if (F.eq(cacheMsg.topologyVersion(), AffinityTopologyVersion.NONE))
+                return new GridFinishedFuture<>();
+
+            return cctx.preloadersStartFuture(cacheMsg.topologyVersion());
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/56fd2725/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 6562257..262d17a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -238,18 +238,20 @@ public class GridCacheSharedContext<K, V> {
     /**
      * @return Compound preloaders start future.
      */
-    public IgniteInternalFuture<Object> preloadersStartFuture() {
+    public IgniteInternalFuture<Object> preloadersStartFuture(AffinityTopologyVersion topVer) {
         if (preloadersStartFut == null) {
             GridCompoundFuture<Object, Object> compound = null;
 
-            for (GridCacheContext<K, V> cacheCtx : cacheContexts()) {
-                IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
+            for (GridCacheContext cacheCtx : cacheContexts()) {
+                if (cacheCtx.startTopologyVersion() != null && cacheCtx.startTopologyVersion().compareTo(topVer) <= 0) {
+                    IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
 
-                if (!startFut.isDone()) {
-                    if (compound == null)
-                        compound = new GridCompoundFuture<>();
+                    if (!startFut.isDone()) {
+                        if (compound == null)
+                            compound = new GridCompoundFuture<>();
 
-                    compound.add(startFut);
+                        compound.add(startFut);
+                    }
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/56fd2725/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 305840b..c8d04ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -672,6 +672,9 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
 
         return S.toString(GridNearOptimisticTxPrepareFuture.class, this,
             "futs", futs,
+            "xidVer", tx.xidVersion(),
+            "nearXidVer", tx.nearXidVersion(),
+            "topVer", tx.topologyVersionSnapshot(),
             "super", super.toString());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/56fd2725/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 4fc6f0c..ad40430 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -183,6 +183,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     private AtomicReference<GridFutureAdapter<IgniteInternalTx>> finFut = new AtomicReference<>();
 
     /** Topology version. */
+    @GridToStringInclude
     protected AtomicReference<AffinityTopologyVersion> topVer = new AtomicReference<>(AffinityTopologyVersion.NONE);
 
     /** Mutex. */


[09/50] [abbrv] ignite git commit: Moved platform cache iterator to Ignite.

Posted by ak...@apache.org.
Moved platform cache iterator to Ignite.


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

Branch: refs/heads/ignite-843
Commit: 136c099bf4ff1ba400e711dbad34216633b9236a
Parents: e428206
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Aug 27 09:20:44 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 27 09:20:44 2015 +0300

----------------------------------------------------------------------
 .../platform/cache/PlatformCacheIterator.java   | 72 ++++++++++++++++++++
 1 file changed, 72 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/136c099b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
new file mode 100644
index 0000000..45e777d
--- /dev/null
+++ b/modules/platform/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.portable.*;
+import org.apache.ignite.internal.processors.platform.*;
+
+import javax.cache.*;
+import java.util.*;
+
+/**
+ * Interop cache iterator.
+ */
+public class PlatformCacheIterator extends PlatformAbstractTarget {
+    /** Operation: next entry. */
+    private static final int OP_NEXT = 1;
+
+    /** Iterator. */
+    private final Iterator<Cache.Entry> iter;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     * @param iter Iterator.
+     */
+    public PlatformCacheIterator(PlatformContext platformCtx, Iterator<Cache.Entry> iter) {
+        super(platformCtx);
+
+        this.iter = iter;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutOp(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_NEXT:
+                if (iter.hasNext()) {
+                    Cache.Entry e = iter.next();
+
+                    assert e != null;
+
+                    writer.writeBoolean(true);
+
+                    writer.writeObjectDetached(e.getKey());
+                    writer.writeObjectDetached(e.getValue());
+                }
+                else
+                    writer.writeBoolean(false);
+
+                break;
+
+            default:
+                throwUnsupported(type);
+        }
+    }
+}


[23/50] [abbrv] ignite git commit: Fixed GridEmbeddedFuture used for async cache operations, added test.

Posted by ak...@apache.org.
Fixed GridEmbeddedFuture used for async cache operations, added test.


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

Branch: refs/heads/ignite-843
Commit: e567f8cb3dd88c3c0a253c4fde06f8ced9b97fde
Parents: c8bc1f9
Author: sboikov <sb...@gridgain.com>
Authored: Thu Aug 27 12:39:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Aug 27 12:41:05 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   6 +-
 .../util/future/GridEmbeddedFuture.java         |  55 +++-
 .../distributed/CacheAsyncOperationsTest.java   | 280 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite3.java       |   2 +
 5 files changed, 342 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e567f8cb/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 7adea2b..54d33e0 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
@@ -4132,9 +4132,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             final IgniteTxLocalAdapter tx0 = tx;
 
             if (fut != null && !fut.isDone()) {
-                IgniteInternalFuture<T> f = new GridEmbeddedFuture<>(fut,
-                    new C2<T, Exception, IgniteInternalFuture<T>>() {
-                        @Override public IgniteInternalFuture<T> apply(T t, Exception e) {
+                IgniteInternalFuture<T> f = new GridEmbeddedFuture(fut,
+                    new IgniteOutClosure<IgniteInternalFuture>() {
+                        @Override public IgniteInternalFuture<T> apply() {
                             return op.op(tx0).chain(new CX1<IgniteInternalFuture<T>, T>() {
                                 @Override public T applyx(IgniteInternalFuture<T> tFut) throws IgniteCheckedException {
                                     try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e567f8cb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index c44b028..9087d20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -567,9 +567,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             IgniteInternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
-                IgniteInternalFuture<T> f = new GridEmbeddedFuture<>(fut,
-                    new C2<T, Exception, IgniteInternalFuture<T>>() {
-                        @Override public IgniteInternalFuture<T> apply(T t, Exception e) {
+                IgniteInternalFuture<T> f = new GridEmbeddedFuture(fut,
+                    new IgniteOutClosure<IgniteInternalFuture>() {
+                        @Override public IgniteInternalFuture<T> apply() {
                             return op.apply();
                         }
                     });

http://git-wip-us.apache.org/repos/asf/ignite/blob/e567f8cb/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
index 4475fae..11b28b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
@@ -68,7 +68,8 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
 
     /**
      * Embeds futures. Specific change order of arguments to avoid conflicts.
-     *  @param embedded Closure.
+     *
+     * @param embedded Embedded future.
      * @param c Closure which runs upon completion of embedded closure and which returns another future.
      */
     public GridEmbeddedFuture(
@@ -200,6 +201,58 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
         });
     }
 
+    /**
+     * @param embedded Embedded future.
+     * @param c Closure to create next future.
+     */
+    public GridEmbeddedFuture(
+        IgniteInternalFuture<B> embedded,
+        final IgniteOutClosure<IgniteInternalFuture<A>> c
+    ) {
+        assert embedded != null;
+        assert c != null;
+
+        this.embedded = embedded;
+
+        embedded.listen(new AL1() {
+            @Override public void applyx(IgniteInternalFuture<B> embedded) {
+                try {
+                    IgniteInternalFuture<A> next = c.apply();
+
+                    if (next == null) {
+                        onDone();
+
+                        return;
+                    }
+
+                    next.listen(new AL2() {
+                        @Override public void applyx(IgniteInternalFuture<A> next) {
+                            try {
+                                onDone(next.get());
+                            }
+                            catch (GridClosureException e) {
+                                onDone(e.unwrap());
+                            }
+                            catch (IgniteCheckedException | RuntimeException e) {
+                                onDone(e);
+                            }
+                            catch (Error e) {
+                                onDone(e);
+
+                                throw e;
+                            }
+                        }
+                    });
+                }
+                catch (Error e) {
+                    onDone(e);
+
+                    throw e;
+                }
+            }
+        });
+    }
+
     /** {@inheritDoc} */
     @Override public boolean cancel() throws IgniteCheckedException {
         return embedded.cancel();

http://git-wip-us.apache.org/repos/asf/ignite/blob/e567f8cb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsTest.java
new file mode 100644
index 0000000..2094d0a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsTest.java
@@ -0,0 +1,280 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+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.junits.common.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class CacheAsyncOperationsTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static volatile CountDownLatch latch;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAsyncOperationsTx() throws Exception {
+        asyncOperations(TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAsyncOperationsAtomic() throws Exception {
+        asyncOperations(ATOMIC);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        CountDownLatch latch0 = latch;
+
+        if (latch0 != null)
+            latch0.countDown();
+
+        latch = null;
+    }
+
+    /**
+     * @param atomicityMode Atomicity mode.
+     * @throws Exception If failed.
+     */
+    public void asyncOperations(CacheAtomicityMode atomicityMode) throws Exception {
+        try (IgniteCache<Integer, Integer> cache = ignite(1).getOrCreateCache(cacheConfiguration(atomicityMode))) {
+            async1(cache);
+
+            async2(cache);
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private void async1(IgniteCache<Integer, Integer> cache) {
+        cache.put(1, 1);
+
+        latch = new CountDownLatch(1);
+
+        IgniteCache<Integer, Integer> asyncCache = cache.withAsync();
+
+        asyncCache.put(0, 0);
+
+        IgniteFuture<?> fut1 = asyncCache.future();
+
+        asyncCache.getAndPut(1, 2);
+
+        IgniteFuture<?> fut2 = asyncCache.future();
+
+        asyncCache.getAndPut(1, 3);
+
+        IgniteFuture<?> fut3 = asyncCache.future();
+
+        assertFalse(fut1.isDone());
+        assertFalse(fut2.isDone());
+        assertFalse(fut3.isDone());
+
+        latch.countDown();
+
+        try {
+            fut1.get();
+
+            fail();
+        }
+        catch (CacheException e) {
+            log.info("Expected error: " + e);
+        }
+
+        assertEquals(1, fut2.get());
+        assertEquals(2, fut3.get());
+
+        assertNull(cache.get(0));
+        assertEquals((Integer)3, cache.get(1));
+    }
+    /**
+     *
+     * @param cache Cache.
+     */
+    private void async2(IgniteCache<Integer, Integer> cache) {
+        cache.put(1, 1);
+
+        latch = new CountDownLatch(1);
+
+        IgniteCache<Integer, Integer> asyncCache = cache.withAsync();
+
+        asyncCache.put(0, 0);
+
+        IgniteFuture<?> fut1 = asyncCache.future();
+
+        asyncCache.put(0, 0);
+
+        IgniteFuture<?> fut2 = asyncCache.future();
+
+        asyncCache.getAndPut(1, 2);
+
+        IgniteFuture<?> fut3 = asyncCache.future();
+
+        asyncCache.put(0, 0);
+
+        IgniteFuture<?> fut4 = asyncCache.future();
+
+        assertFalse(fut1.isDone());
+        assertFalse(fut2.isDone());
+        assertFalse(fut3.isDone());
+        assertFalse(fut4.isDone());
+
+        latch.countDown();
+
+        try {
+            fut1.get();
+
+            fail();
+        }
+        catch (CacheException e) {
+            log.info("Expected error: " + e);
+        }
+
+        try {
+            fut2.get();
+
+            fail();
+        }
+        catch (CacheException e) {
+            log.info("Expected error: " + e);
+        }
+
+        assertEquals(1, fut3.get());
+
+        try {
+            fut4.get();
+
+            fail();
+        }
+        catch (CacheException e) {
+            log.info("Expected error: " + e);
+        }
+
+        assertNull(cache.get(0));
+        assertEquals((Integer)2, cache.get(1));
+    }
+
+    /**
+     * @param atomicityMode Atomicity mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(CacheAtomicityMode atomicityMode) {
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setWriteThrough(true);
+        ccfg.setCacheStoreFactory(new StoreFactory());
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    private static class StoreFactory implements Factory<TestStore> {
+        /** {@inheritDoc} */
+        @Override public TestStore create() {
+            return new TestStore();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestStore extends CacheStoreAdapter<Integer, Integer> {
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry) {
+            CountDownLatch latch0 = latch;
+
+            if (latch0 != null)
+                U.awaitQuiet(latch0);
+
+            Integer key = entry.getKey();
+
+            if (key.equals(0)) {
+                System.out.println(Thread.currentThread().getName() + ": fail operation for key: " + key);
+
+                throw new CacheWriterException("Test error.");
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e567f8cb/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index 5947d33..c20e901 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -133,6 +133,8 @@ public class IgniteCacheTestSuite3 extends TestSuite {
 
         suite.addTestSuite(IgniteTxGetAfterStopTest.class);
 
+        suite.addTestSuite(CacheAsyncOperationsTest.class);
+
         return suite;
     }
 }