You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/10/23 09:46:47 UTC

[1/7] ignite git commit: Fixing NIO thread death if communication exception is occurred.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1607 dce9bd592 -> 61df8c26d


Fixing NIO thread death if communication exception is occurred.


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

Branch: refs/heads/ignite-1607
Commit: 473cb9e70a2f74561435edbda522d7b5800ac7d2
Parents: ff96888
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Oct 20 17:45:02 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Oct 20 17:45:02 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioServer.java |   2 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java | 105 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 3 files changed, 109 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/473cb9e7/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 82fb1bb..1824339 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -1443,7 +1443,7 @@ public class GridNioServer<T> {
                     // This exception will be handled in bodyInternal() method.
                     throw e;
                 }
-                catch (IOException e) {
+                catch (Exception e) {
                     if (!closed)
                         U.warn(log, "Failed to process selector key (will close): " + ses, e);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/473cb9e7/modules/core/src/test/java/org/apache/ignite/internal/util/nio/IgniteExceptionInNioWorkerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/IgniteExceptionInNioWorkerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/IgniteExceptionInNioWorkerSelfTest.java
new file mode 100644
index 0000000..9961833
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/IgniteExceptionInNioWorkerSelfTest.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.util.nio;
+
+import java.util.UUID;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridTopic;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteExceptionInNioWorkerSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final int GRID_CNT = 4;
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration("cache");
+
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBrokenMessage() throws Exception {
+        startGrids(GRID_CNT);
+
+        try {
+            IgniteKernal kernal = (IgniteKernal)ignite(0);
+
+            UUID nodeId = ignite(1).cluster().localNode().id();
+
+            // This should trigger a failure in a NIO thread.
+            kernal.context().io().send(nodeId, GridTopic.TOPIC_CACHE.topic("cache"), new BrokenMessage(), (byte)0);
+
+            for (int i = 0; i < 100; i++)
+                ignite(0).cache("cache").put(i, i);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class BrokenMessage extends AffinityTopologyVersion {
+        /** */
+        private boolean fail = true;
+
+        /** {@inheritDoc} */
+        @Override public byte directType() {
+            if (fail) {
+                fail = false;
+
+                return (byte)242;
+            }
+
+            return super.directType();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/473cb9e7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 1a9913f..0e5894d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest
 import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest;
 import org.apache.ignite.internal.processors.service.ClosureServiceClientsNodesTest;
 import org.apache.ignite.internal.product.GridProductVersionSelfTest;
+import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.messaging.GridMessagingNoPeerClassLoadingSelfTest;
 import org.apache.ignite.messaging.GridMessagingSelfTest;
@@ -111,6 +112,8 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
         suite.addTestSuite(IgniteDaemonNodeMarshallerCacheTest.class);
 
+        suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
+
         return suite;
     }
 }
\ No newline at end of file


[6/7] ignite git commit: ignite-1607 WIP

Posted by sb...@apache.org.
ignite-1607 WIP


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

Branch: refs/heads/ignite-1607
Commit: 12811ecdb8b5f2e32d50015f44504e0279063f92
Parents: dce9bd5
Author: sboikov <sb...@gridgain.com>
Authored: Fri Oct 23 10:42:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Oct 23 10:43:41 2015 +0300

----------------------------------------------------------------------
 .../cache/SerializableTransactionsLockTest.java | 547 -------------------
 .../yardstick/config/benchmark-1607.properties  |  75 ---
 .../config/benchmark-multicast.properties       |   5 +-
 3 files changed, 4 insertions(+), 623 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/12811ecd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/SerializableTransactionsLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/SerializableTransactionsLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/SerializableTransactionsLockTest.java
deleted file mode 100644
index 1ac1c9f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/SerializableTransactionsLockTest.java
+++ /dev/null
@@ -1,547 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.util.future.GridCompoundFuture;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteReducer;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.thread.IgniteThreadPoolExecutor;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class SerializableTransactionsLockTest extends GridCommonAbstractTest {
-    /** */
-    private static final boolean TRY_LOCK = false;
-
-    /** */
-    private List<Node> nodes = new ArrayList<>();
-
-    /** */
-    private AtomicLong txIdGen = new AtomicLong();
-
-    /** {@inheritDoc} */
-    @Override protected long getTestTimeout() {
-        return 6 * 60_000;
-    }
-
-    /**
-     * @param cnt Number of nodes.
-     */
-    private void createNodes(int cnt) {
-        for (int i = 0; i < cnt; i++)
-            nodes.add(new Node("n-" + i));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testConflict() throws Exception {
-        createNodes(5);
-
-        final AtomicBoolean stop = new AtomicBoolean();
-
-        final AtomicInteger totalCnt = new AtomicInteger();
-
-        final AtomicInteger rollbackCnt = new AtomicInteger();
-
-        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                int iter = 0;
-
-                List<Integer> keys = new ArrayList<>();
-
-                for (int i = 0; i < 100; i++)
-                    keys.add(i);
-
-                while (!stop.get()) {
-                    Collections.shuffle(keys);
-
-                    boolean commit = executeTransaction(txIdGen.incrementAndGet(), keys);
-
-                    totalCnt.incrementAndGet();
-
-                    if (!commit)
-                        rollbackCnt.incrementAndGet();
-
-                    if (iter % 1000 == 0)
-                        log.info("Iteration: " + iter);
-
-                    iter++;
-                }
-
-                return null;
-            }
-        }, 5, "tx-thread");
-
-        Thread.sleep(5_000);
-
-        stop.set(true);
-
-        fut.get();
-
-        log.info("Result [total=" + totalCnt.get() +
-            ", rollback=" + rollbackCnt.get() +
-            ", rollbackPercent=" + rollbackCnt.get() / (float)totalCnt.get() * 100 + "%]");
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoDeadlock() throws Exception {
-        createNodes(10);
-
-        final AtomicBoolean stop = new AtomicBoolean();
-
-        final AtomicInteger totalCnt = new AtomicInteger();
-
-        final AtomicInteger rollbackCnt = new AtomicInteger();
-
-        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                int iter = 0;
-
-                while (!stop.get()) {
-                    int keysNum = rnd.nextInt(1, 30);
-
-                    Set<Integer> keys = new LinkedHashSet<>();
-
-                    while (keys.size() < keysNum)
-                        keys.add(rnd.nextInt(100));
-
-                    boolean commit = executeTransaction(txIdGen.incrementAndGet(), keys);
-
-                    totalCnt.incrementAndGet();
-
-                    if (!commit)
-                        rollbackCnt.incrementAndGet();
-
-                    if (iter % 5000 == 0)
-                        log.info("Iteration: " + iter);
-
-                    iter++;
-                }
-
-                return null;
-            }
-        }, 30, "tx-thread");
-
-        Thread.sleep(5 * 60_000);
-
-        stop.set(true);
-
-        fut.get();
-
-        log.info("Result [total=" + totalCnt.get() +
-            ", rollback=" + rollbackCnt.get() +
-            ", rollbackPercent=" + rollbackCnt.get() / (float)totalCnt.get() * 100 + "%]");
-    }
-
-    /**
-     * @param txId Transaction ID.
-     * @param keys Keys.
-     * @return {@code True} if transaction acquired all locks.
-     * @throws Exception If failed.
-     */
-    private boolean executeTransaction(Long txId, Collection<Integer> keys) throws Exception {
-        Map<Node, Set<Integer>> txMap = new HashMap<>();
-
-        for (Integer key : keys) {
-            Node node =  map(key);
-
-            Set<Integer> nodeKeys = txMap.get(node);
-
-            if (nodeKeys == null)
-                txMap.put(node, nodeKeys = new HashSet<>());
-
-            nodeKeys.add(key);
-        }
-
-        Map<Node, IgniteInternalFuture<Boolean>> futs = new HashMap<>();
-
-        for (Map.Entry<Node, Set<Integer>> e : txMap.entrySet()) {
-            Node node = e.getKey();
-
-            futs.put(node, node.prepare(txId, e.getValue()));
-        }
-
-        boolean prepared = true;
-
-        for (Map.Entry<Node, IgniteInternalFuture<Boolean>> e : futs.entrySet()) {
-            boolean nodePrepared = e.getValue().get();
-
-            if (!nodePrepared) {
-                prepared = false;
-
-                txMap.remove(e.getKey());
-            }
-        }
-
-        GridCompoundFuture<Void, Void> finishFut = new GridCompoundFuture<>();
-
-        for (Map.Entry<Node, Set<Integer>> e : txMap.entrySet()) {
-            Node node = e.getKey();
-
-            finishFut.add(node.finish(txId, e.getValue(), prepared));
-        }
-
-        finishFut.markInitialized();
-
-        finishFut.get();
-
-        return prepared;
-    }
-
-    /**
-     * @param key Key.
-     * @return Node.
-     */
-    private Node map(Integer key) {
-        int idx = U.safeAbs(key.hashCode()) % nodes.size();
-
-        return nodes.get(idx);
-    }
-
-    /**
-     *
-     */
-    static class Node {
-        /** */
-        private ConcurrentHashMap<Integer, Entry> map = new ConcurrentHashMap<>();
-
-        /** */
-        private final String id;
-
-        /** */
-        IgniteThreadPoolExecutor execSvc;
-
-        /**
-         * @param id ID.
-         */
-        public Node(String id) {
-            this.id = id;
-
-            execSvc = new IgniteThreadPoolExecutor(
-                "pool-" + id,
-                IgniteConfiguration.DFLT_SYSTEM_CORE_THREAD_CNT,
-                IgniteConfiguration.DFLT_SYSTEM_CORE_THREAD_CNT,
-                0,
-                new LinkedBlockingQueue<Runnable>(Integer.MAX_VALUE));
-        }
-
-        /**
-         * @param txId Transaction ID.
-         * @param keys Keys.
-         * @return Prepare future.
-         */
-        public IgniteInternalFuture<Boolean> prepare(final Long txId, final Collection<Integer> keys) {
-            final GridCompoundFuture<Boolean, Boolean> fut =
-                new GridCompoundFuture<>(new IgniteReducer<Boolean, Boolean>() {
-                    /** */
-                    private volatile boolean res = true;
-
-                    @Override public boolean collect(Boolean prepared) {
-                        if (!prepared)
-                            res = false;
-
-                        return true;
-                    }
-
-                    @Override public Boolean reduce() {
-                        return res;
-                    }
-                }
-            );
-
-            execSvc.submit(new Runnable() {
-                @Override public void run() {
-                    List<Entry> locked = new ArrayList<>(keys.size());
-
-                    for (Integer key : keys) {
-                        Entry e = entry(key);
-
-                        IgniteInternalFuture<Boolean> keyFut = e.lock(txId);
-
-                        if (keyFut == null) {
-                            for (Entry l : locked)
-                                l.unlock(txId);
-
-                            fut.onDone(false);
-
-                            return;
-                        }
-
-                        locked.add(e);
-
-                        fut.add(keyFut);
-                    }
-
-                    fut.markInitialized();
-
-                    for (Entry e : locked)
-                        e.txReady(txId);
-                }
-            });
-
-            return fut;
-        }
-
-        /**
-         * @param txId Transaction ID.
-         * @param keys Transaction keys
-         * @param commit Commit flag.
-         * @return Commit future.
-         */
-        public IgniteInternalFuture<Void> finish(final Long txId, final Collection<Integer> keys, boolean commit) {
-            final GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
-
-            execSvc.submit(new Runnable() {
-                @Override public void run() {
-                    for (Integer key : keys) {
-                        Entry e = entry(key);
-
-                        e.unlock(txId);
-                    }
-
-                    fut.onDone();
-                }
-            });
-
-            return fut;
-        }
-
-        /**
-         * @param key Key.
-         * @return Entry.
-         */
-        private Entry entry(Integer key) {
-            Entry e = map.get(key);
-
-            if (e == null) {
-                Entry old = map.putIfAbsent(key, e = new Entry(key));
-
-                if (old != null)
-                    e = old;
-            }
-
-            return e;
-        }
-
-        /**
-         * @return Node ID.
-         */
-        String id() {
-            return id;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            Node node = (Node)o;
-
-            return id.equals(node.id);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return id.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "Node [id=" + id + ']';
-        }
-    }
-
-    /**
-     *
-     */
-    static class Entry {
-        /** */
-        private final Deque<Candidate> cands = new ArrayDeque<>();
-
-        /** */
-        private final Integer key;
-
-        /**
-         * @param key Key.
-         */
-        public Entry(Integer key) {
-            this.key = key;
-        }
-
-        /**
-         * @return Key.
-         */
-        Integer key() {
-            return key;
-        }
-
-        /**
-         *
-         */
-        static class Candidate {
-            /** */
-            private Long txId;
-
-            /** */
-            private GridFutureAdapter<Boolean> fut;
-
-            /** */
-            private boolean ready;
-
-            /**
-             * @param txId Transaction ID.
-             * @param fut Future.
-             */
-            public Candidate(Long txId, GridFutureAdapter<Boolean> fut) {
-                this.txId = txId;
-                this.fut = fut;
-            }
-        }
-
-        /**
-         * @param txId Transaction ID.
-         * @return Future.
-         */
-        synchronized IgniteInternalFuture<Boolean> lock(Long txId) {
-            Candidate last = cands.peekLast();
-
-            if (TRY_LOCK) {
-                if (last != null)
-                    return null;
-            }
-            else {
-                if (last != null && last.txId > txId)
-                    return null;
-            }
-
-            GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
-
-            cands.addLast(new Candidate(txId, fut));
-
-            return fut;
-        }
-
-        /**
-         * @param txId Transaction ID.
-         */
-        void txReady(Long txId) {
-            Candidate owner;
-
-            synchronized (this) {
-                Candidate cand = candidate(txId);
-
-                cand.ready = true;
-
-                owner = assignNewOwner();
-            }
-
-            if (owner != null)
-                owner.fut.onDone(true);
-        }
-
-        /**
-         * @param txId Transaction ID.
-         */
-        void unlock(Long txId) {
-             Candidate owner;
-
-             synchronized (this) {
-                 Candidate cand = candidate(txId);
-
-                 cands.remove(cand);
-
-                 owner = assignNewOwner();
-             }
-
-             if (owner != null)
-                 owner.fut.onDone(true);
-        }
-
-        /**
-         * @return New owner.
-         */
-        @Nullable private Candidate assignNewOwner() {
-            Candidate cand = cands.peekFirst();
-
-            if (cand != null && cand.ready)
-                return cand;
-
-            return cand;
-        }
-
-        /**
-         * @param txId Transaction ID.
-         * @return Candidate for transaction.
-         */
-        private Candidate candidate(Long txId) {
-            Candidate txCand = null;
-
-            for (Candidate cand : cands) {
-                if (cand.txId.equals(txId)) {
-                    txCand = cand;
-
-                    break;
-                }
-            }
-
-            assert txCand != null;
-
-            return txCand;
-        }
-
-        /** {@inheritDoc} */
-        public String toString() {
-            return S.toString(Entry.class, this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/12811ecd/modules/yardstick/config/benchmark-1607.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-1607.properties b/modules/yardstick/config/benchmark-1607.properties
deleted file mode 100644
index a82e470..0000000
--- a/modules/yardstick/config/benchmark-1607.properties
+++ /dev/null
@@ -1,75 +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.
-
-#
-# Contains all multicast benchmarks
-#
-
-# JVM options.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
-
-# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" \
--Xloggc:./gc.log \
--XX:+PrintGCDetails \
--verbose:gc \
--XX:+UseParNewGC \
--XX:+UseConcMarkSweepGC \
--XX:+UseTLAB \
--XX:NewSize=128m \
--XX:MaxNewSize=128m \
--XX:MaxTenuringThreshold=0 \
--XX:SurvivorRatio=1024 \
--XX:+UseCMSInitiatingOccupancyOnly \
--XX:CMSInitiatingOccupancyFraction=60 \
-"
-#Ignite version
-ver="RELEASE-"
-
-# List of default probes.
-# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
-BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe
-
-# Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
-
-# Restart servers for each benchmark.
-RESTART_SERVERS=true
-
-# Probe point writer class name.
-# BENCHMARK_WRITER=
-
-# Comma-separated list of the hosts to run BenchmarkServers on.
-SERVER_HOSTS=localhost,localhost,localhost
-
-# Comma-separated list of the hosts to run BenchmarkDrivers on.
-DRIVER_HOSTS=localhost
-
-# Remote username.
-# REMOTE_USER=
-
-# Number of nodes, used to wait for the specified number of nodes to start.
-nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
-
-# Run configuration which contains all benchmarks.
-# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
-CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putall,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putall-serializable,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteAccountTxBenchmark -sn IgniteNode -ds ${ver}tx-account,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteAccountSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-account-serializable,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC --keysPerThread -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putall-keys-per-thread,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC --keysPerThread -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putall-keys-per-thread-serializable\
-"

http://git-wip-us.apache.org/repos/asf/ignite/blob/12811ecd/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index 82bf766..82fc3f8 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -86,5 +86,8 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxOffHeapValuesBenchmark -sn IgniteNode -ds ${ver}tx-put-offheap-val-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-offheap-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-join-offheap-1-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-put-offheap-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutOffHeapBenchmark -sn IgniteNode -ds ${ver}sql-query-put-offheap-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 100 -dn IgnitePutAllBenchmark -sn IgniteNode -ds ${ver}atomic-putAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup\
 "


[2/7] ignite git commit: 1.4.1-p1-SNAPSHOT

Posted by sb...@apache.org.
1.4.1-p1-SNAPSHOT


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

Branch: refs/heads/ignite-1607
Commit: 707aacc0bed77237609608ba5d9251c62920bd16
Parents: 473cb9e
Author: Ignite Teamcity <ig...@apache.org>
Authored: Tue Oct 20 19:48:17 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Tue Oct 20 19:48:17 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                                | 2 +-
 examples/schema-import/pom.xml                                  | 5 ++---
 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/jms11/pom.xml                                           | 2 +-
 modules/jta/pom.xml                                             | 2 +-
 modules/kafka/pom.xml                                           | 2 +-
 modules/log4j/pom.xml                                           | 2 +-
 modules/log4j2/pom.xml                                          | 2 +-
 modules/mesos/pom.xml                                           | 2 +-
 modules/platform/pom.xml                                        | 2 +-
 modules/platform/src/main/cpp/common/configure.ac               | 2 +-
 modules/platform/src/main/cpp/core-test/configure.ac            | 2 +-
 modules/platform/src/main/cpp/core/configure.ac                 | 2 +-
 .../main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs   | 4 ++--
 .../src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs    | 4 ++--
 .../Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs | 4 ++--
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs  | 4 ++--
 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 +-
 modules/zookeeper/pom.xml                                       | 2 +-
 pom.xml                                                         | 2 +-
 53 files changed, 58 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index a2ac452..8a23fdc 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/examples/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/examples/schema-import/pom.xml b/examples/schema-import/pom.xml
index 46bc83e..ba4e38d 100644
--- a/examples/schema-import/pom.xml
+++ b/examples/schema-import/pom.xml
@@ -20,8 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -36,7 +35,7 @@
     </properties>
 
     <artifactId>ignite-schema-import-demo</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index a080b57..9b8c865 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 1af8c00..b1d3478 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,7 +31,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 63d454d..9b65d5c 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 74590ec..7cae06b 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index e735804..fa11d38 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index a39da2f..35db837 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 13d6fd4..d646310 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/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 c1c2d0f..fdf6414 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.4.1-SNAPSHOT
+ignite.version=1.4.1-p1-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index b48859b..261d4ef 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/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 f863bed..183fccc 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 0b4f92f..a0ae635 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index b3fbb32..dc2e2e5 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 1330cf5..312c8c7 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 9fdfd99..916e725 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 2d6d893..a6f808e 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 4bcadc8..a3e2a31 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 0b2c48b..16b707e 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/jms11/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jms11/pom.xml b/modules/jms11/pom.xml
index 1b8f67d..69e1c11 100644
--- a/modules/jms11/pom.xml
+++ b/modules/jms11/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jms11</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 7965dd9..71b1c2c 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index ed192ab..8a12ff0 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index c3c4a84..d028e7a 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/log4j2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j2/pom.xml b/modules/log4j2/pom.xml
index 0628b47..b89bed2 100644
--- a/modules/log4j2/pom.xml
+++ b/modules/log4j2/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j2</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 07bd13d..8a65d8e 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/pom.xml
----------------------------------------------------------------------
diff --git a/modules/platform/pom.xml b/modules/platform/pom.xml
index f18b05b..ed2979f 100644
--- a/modules/platform/pom.xml
+++ b/modules/platform/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-platform</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/main/cpp/common/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/common/configure.ac b/modules/platform/src/main/cpp/common/configure.ac
index 3f7c8a2..6d87b24 100644
--- a/modules/platform/src/main/cpp/common/configure.ac
+++ b/modules/platform/src/main/cpp/common/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite JNI bridge for C++], [1.4.1], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
+AC_INIT([Apache Ignite JNI bridge for C++], [1.4.1.1], [dev@ignite.apache.org], [ignite-common], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/main/cpp/core-test/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/core-test/configure.ac b/modules/platform/src/main/cpp/core-test/configure.ac
index c72e6bd..36a3a88 100644
--- a/modules/platform/src/main/cpp/core-test/configure.ac
+++ b/modules/platform/src/main/cpp/core-test/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++ Test], [1.4.1], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++ Test], [1.4.1.1], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/main/cpp/core/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/cpp/core/configure.ac b/modules/platform/src/main/cpp/core/configure.ac
index 4a3a03a..34f2fb1 100644
--- a/modules/platform/src/main/cpp/core/configure.ac
+++ b/modules/platform/src/main/cpp/core/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++], [1.4.1], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++], [1.4.1.1], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 AC_CONFIG_SRCDIR(src)
 
 AC_CANONICAL_SYSTEM

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 8e653e3..7395cc1 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")]
 
-[assembly: AssemblyVersion("1.4.1")]
-[assembly: AssemblyFileVersion("1.4.1")]
+[assembly: AssemblyVersion("1.4.1.1")]
+[assembly: AssemblyFileVersion("1.4.1.1")]
 
 [assembly: CLSCompliant(true)]
 [assembly: InternalsVisibleTo("Apache.Ignite")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
index 7cf47c3..e81eb12 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("0f9702ec-da7d-4ce5-b4b7-73310c885355")]
 
-[assembly: AssemblyVersion("1.4.1")]
-[assembly: AssemblyFileVersion("1.4.1")]
\ No newline at end of file
+[assembly: AssemblyVersion("1.4.1.1")]
+[assembly: AssemblyFileVersion("1.4.1.1")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
index ba4b794..8f2f2f9 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
@@ -45,5 +45,5 @@ using System.Runtime.InteropServices;
 // You can specify all the values or you can default the Build and Revision Numbers 
 // by using the '*' as shown below:
 // [assembly: AssemblyVersion("1.0.*")]
-[assembly: AssemblyVersion("1.4.1")]
-[assembly: AssemblyFileVersion("1.4.1")]
+[assembly: AssemblyVersion("1.4.1.1")]
+[assembly: AssemblyFileVersion("1.4.1.1")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index f474b03..ede7b55 100644
--- a/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/test/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -31,5 +31,5 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("de8dd5cc-7c7f-4a09-80d5-7086d9416a7b")]
 
-[assembly: AssemblyVersion("1.4.1")]
-[assembly: AssemblyFileVersion("1.4.1")]
\ No newline at end of file
+[assembly: AssemblyVersion("1.4.1.1")]
+[assembly: AssemblyFileVersion("1.4.1.1")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 8156887..f560daf 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 42fb9b9..efc1936 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 982fcaf..bbe0fdb 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index e64059a..17bfd85 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 0ba8597..b0e1bb7 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 5cdf791..0773716 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 6c44006..4cb015f 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 1fb2753..332e70e 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 4ed1fd7..f72ef35 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 7922f53..b7b719b 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 5e3465e..fb543b9 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 7dd0431..a90a21e 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/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 a0edeaf..55b4f98 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index d6f373a..cbc2226 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 04142fa..c562939 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.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 4432d81..5b55e82 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 3f3774a..768c3ab 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 079a173..86827e9 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index ba10101..497d119 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-zookeeper</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <url>http://ignite.apache.org</url>
 
     <properties>

http://git-wip-us.apache.org/repos/asf/ignite/blob/707aacc0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6a17668..cfb4838 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.4.1-SNAPSHOT</version>
+    <version>1.4.1-p1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[4/7] ignite git commit: ignite-1718: proper type name extraction in portable context for classes that ends on '$' sign

Posted by sb...@apache.org.
ignite-1718: proper type name extraction in portable context for classes that ends on '$' sign


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

Branch: refs/heads/ignite-1607
Commit: e9524cea121ce70338bb9bf1a5f622acc445d5f0
Parents: 91e31e9
Author: Andrey Gura <ag...@gridgain.com>
Authored: Thu Oct 22 11:05:02 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Oct 22 11:05:02 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/portable/PortableContext.java       | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e9524cea/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 1ad42ab..e9cccf0 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
@@ -547,7 +547,6 @@ public class PortableContext implements Externalizable {
 
         try {
             registered = marshCtx.registerClass(typeId, cls);
-
         }
         catch (IgniteCheckedException e) {
             throw new PortableException("Failed to register class.", e);
@@ -891,10 +890,11 @@ public class PortableContext implements Externalizable {
 
         int idx = clsName.lastIndexOf('$');
 
-        String typeName;
-
-        if (idx >= 0) {
-            typeName = clsName.substring(idx + 1);
+        if (idx == clsName.length() - 1)
+            // This is a regular (not inner) class name that ends with '$'. Common use case for Scala classes.
+            idx = -1;
+        else if (idx >= 0) {
+            String typeName = clsName.substring(idx + 1);
 
             try {
                 Integer.parseInt(typeName);
@@ -1147,4 +1147,4 @@ public class PortableContext implements Externalizable {
             return registered;
         }
     }
-}
\ No newline at end of file
+}


[5/7] ignite git commit: Fixed exception handling in NIO worker.

Posted by sb...@apache.org.
Fixed exception handling in NIO worker.


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

Branch: refs/heads/ignite-1607
Commit: 0bc1d6f42e637e5bfc45aa99fdcb68fc7896ef4e
Parents: e9524ce f49deb2
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Oct 22 15:47:03 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 22 15:47:03 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioServer.java |   2 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java | 105 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 3 files changed, 109 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[3/7] ignite git commit: .Net: fix InternalsVisibleTo

Posted by sb...@apache.org.
.Net: fix InternalsVisibleTo


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

Branch: refs/heads/ignite-1607
Commit: f49deb28121c718207bbc9b92acf4fdb8103012c
Parents: 707aacc
Author: ptupitsyn <pt...@gridgain.com>
Authored: Wed Oct 21 12:37:57 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Wed Oct 21 12:37:57 2015 +0300

----------------------------------------------------------------------
 .../src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f49deb28/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 7395cc1..448e195 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -37,10 +37,9 @@ using System.Runtime.InteropServices;
 [assembly: AssemblyFileVersion("1.4.1.1")]
 
 [assembly: CLSCompliant(true)]
-[assembly: InternalsVisibleTo("Apache.Ignite")]
-
 #if !EXCLUDE_TESTS
 
+[assembly: InternalsVisibleTo("Apache.Ignite")]
 [assembly: InternalsVisibleTo("Apache.Ignite.Core.Tests")]
 
 #endif
\ No newline at end of file


[7/7] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1607

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


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

Branch: refs/heads/ignite-1607
Commit: 61df8c26d67e98d8e2d96b31ed7b070799f0700f
Parents: 12811ec 0bc1d6f
Author: sboikov <sb...@gridgain.com>
Authored: Fri Oct 23 10:44:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Oct 23 10:44:24 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      |  12 +--
 .../ignite/internal/util/nio/GridNioServer.java |   2 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java | 105 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 4 files changed, 115 insertions(+), 7 deletions(-)
----------------------------------------------------------------------