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/05/06 14:07:00 UTC

[01/36] incubator-ignite git commit: ignite-646

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-695 eb9868021 -> 6ea8f92d4


ignite-646


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

Branch: refs/heads/ignite-695
Commit: 4843b661a6ae61288aaa1fb483d5f8f68c3729c6
Parents: 5049704
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 17 19:49:14 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 17 19:49:14 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  22 +--
 .../processors/cache/GridCacheMessage.java      |   8 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  19 ++
 .../dht/atomic/GridDhtAtomicCache.java          |  31 +++-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   8 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   2 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |  18 +-
 .../distributed/near/GridNearGetResponse.java   |   8 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   | 182 +++++++++++++++++++
 9 files changed, 273 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/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 6fefdfd..56ee65e 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
@@ -282,13 +282,13 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
         }
         catch (Throwable e) {
-            if (X.hasCause(e, ClassNotFoundException.class))
-                U.error(log, "Failed to process message (note that distributed services " +
-                    "do not support peer class loading, if you deploy distributed service " +
-                    "you should have all required classes in CLASSPATH on all nodes in topology) " +
-                    "[senderId=" + nodeId + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
-            else
-                U.error(log, "Failed to process message [senderId=" + nodeId + ']', e);
+//            if (X.hasCause(e, ClassNotFoundException.class))
+//                U.error(log, "Failed to process message (note that distributed services " +
+//                    "do not support peer class loading, if you deploy distributed service " +
+//                    "you should have all required classes in CLASSPATH on all nodes in topology) " +
+//                    "[senderId=" + nodeId + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
+//            else
+            U.error(log, "Failed to process message [senderId=" + nodeId + ", messageType=" + cacheMsg.getClass() + ']', e);
         }
         finally {
             if (depEnabled)
@@ -738,11 +738,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             cacheMsg.finishUnmarshal(cctx, cctx.deploy().globalLoader());
         }
         catch (IgniteCheckedException e) {
-            if (cacheMsg.ignoreClassErrors() && X.hasCause(e, InvalidClassException.class,
-                    ClassNotFoundException.class, NoClassDefFoundError.class, UnsupportedClassVersionError.class))
+//            if (cacheMsg.ignoreClassErrors() && X.hasCause(e, InvalidClassException.class,
+//                    ClassNotFoundException.class, NoClassDefFoundError.class, UnsupportedClassVersionError.class))
                 cacheMsg.onClassError(e);
-            else
-                throw e;
+//            else
+//                throw e;
         }
         catch (Error e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, NoClassDefFoundError.class,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index fefd582..5432c90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -60,7 +60,7 @@ public abstract class GridCacheMessage implements Message {
 
     /** */
     @GridDirectTransient
-    private Exception err;
+    private IgniteCheckedException err;
 
     /** */
     @GridDirectTransient
@@ -115,14 +115,14 @@ public abstract class GridCacheMessage implements Message {
      *
      * @param err Error.
      */
-    public void onClassError(Exception err) {
+    public void onClassError(IgniteCheckedException err) {
         this.err = err;
     }
 
     /**
-     * @return Error set via {@link #onClassError(Exception)} method.
+     * @return Error set via {@link #onClassError(IgniteCheckedException)} method.
      */
-    public Exception classError() {
+    public IgniteCheckedException classError() {
         return err;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 6e6cb04..e5312bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -606,6 +606,25 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     protected void processNearGetRequest(final UUID nodeId, final GridNearGetRequest req) {
         assert ctx.affinityNode();
 
+        if (req.classError() != null) {
+            GridNearGetResponse res = new GridNearGetResponse(ctx.cacheId(),
+                req.futureId(),
+                req.miniId(),
+                req.version());
+
+            res.error(req.classError());
+
+            try {
+                ctx.io().send(nodeId, res, ctx.ioPolicy());
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId +
+                    ",req=" + req + ", res=" + res + ']', e);
+            }
+
+            return;
+        }
+
         long ttl = req.accessTtl();
 
         final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.forAccess(ttl);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/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 6984238..b46cd57 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
@@ -2329,7 +2329,18 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         req.nodeId(ctx.localNodeId());
 
-        updateAllAsyncInternal(nodeId, req, updateReplyClos);
+        if (req.classError() != null) {
+            GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
+                nodeId,
+                req.futureVersion());
+
+            res.onError(req.classError());
+
+            sendNearUpdateReply(nodeId, res);
+        }
+        else {
+            updateAllAsyncInternal(nodeId, req, updateReplyClos);
+        }
     }
 
     /**
@@ -2365,6 +2376,24 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         // Always send update reply.
         GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(ctx.cacheId(), req.futureVersion());
 
+        if (req.classError() != null) {
+            res.onError(req.classError());
+
+            try {
+                ctx.io().send(nodeId, res, ctx.ioPolicy());
+            }
+            catch (ClusterTopologyCheckedException ignored) {
+                U.warn(log, "Failed to send DHT atomic update response to node because it left grid: " +
+                    req.nodeId());
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send DHT atomic update response (did node leave grid?) [nodeId=" + nodeId +
+                    ", req=" + req + ']', e);
+            }
+
+            return;
+        }
+
         Boolean replicate = ctx.isDrEnabled();
 
         boolean intercept = req.forceTransformBackups() && ctx.config().getInterceptor() != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
index d0a7620..c5b5a37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java
@@ -88,6 +88,14 @@ public class GridDhtAtomicUpdateResponse extends GridCacheMessage implements Gri
     }
 
     /**
+     * Sets update error.
+     * @param err
+     */
+    public void onError(IgniteCheckedException err){
+        this.err = err;
+    }
+
+    /**
      * @return Gets update error.
      */
     public IgniteCheckedException error() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 974a197..159da99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -335,7 +335,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             updateNear(singleReq, res);
 
             if (res.error() != null)
-                onDone(addFailedKeys(res.failedKeys(), res.error()));
+                onDone(res.failedKeys() != null ? addFailedKeys(res.failedKeys(), res.error()) : res.error());
             else {
                 if (op == TRANSFORM) {
                     if (ret != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 01d5722..773b847 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -137,9 +137,17 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     }
 
     /**
+     * Sets update error.
+     * @param err
+     */
+    public void onError(IgniteCheckedException err){
+        this.err = err;
+    }
+
+    /**
      * @return Update error, if any.
      */
-    public Throwable error() {
+    public IgniteCheckedException error() {
         return err;
     }
 
@@ -335,10 +343,12 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param e Error cause.
      */
     public synchronized void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e) {
-        if (failedKeys == null)
-            failedKeys = new ArrayList<>(keys.size());
+        if (keys != null) {
+            if (failedKeys == null)
+                failedKeys = new ArrayList<>(keys.size());
 
-        failedKeys.addAll(keys);
+            failedKeys.addAll(keys);
+        }
 
         if (err == null)
             err = new IgniteCheckedException("Failed to update keys on primary node.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
index 57652bd..73d877a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java
@@ -65,7 +65,7 @@ public class GridNearGetResponse extends GridCacheMessage implements GridCacheDe
 
     /** Error. */
     @GridDirectTransient
-    private Throwable err;
+    private IgniteCheckedException err;
 
     /** Serialized error. */
     private byte[] errBytes;
@@ -152,20 +152,20 @@ public class GridNearGetResponse extends GridCacheMessage implements GridCacheDe
      * @return Topology version if this response has invalid partitions.
      */
     @Override public AffinityTopologyVersion topologyVersion() {
-        return topVer;
+        return topVer != null ? topVer : super.topologyVersion();
     }
 
     /**
      * @return Error.
      */
-    public Throwable error() {
+    public IgniteCheckedException error() {
         return err;
     }
 
     /**
      * @param err Error.
      */
-    public void error(Throwable err) {
+    public void error(IgniteCheckedException err) {
         this.err = err;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4843b661/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
new file mode 100644
index 0000000..d21c219
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -0,0 +1,182 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Check behavior on exception while unmarshalling key
+ */
+public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTest {
+    /** Allows to change behavior of readExternal method */
+    private static AtomicInteger nodeCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CacheAtomicWriteOrderMode.PRIMARY;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (gridName.endsWith("0"))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** Test key 1 */
+    public static class TestKey implements Externalizable {
+        /** Test key 1 */
+        public TestKey(String field) {
+            this.field = field;
+        }
+
+        /** Test key 1 */
+        public TestKey() {
+        }
+
+        /** field */
+        private String field;
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestKey key = (TestKey)o;
+
+            return !(field != null ? !field.equals(key.field) : key.field != null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return field != null ? field.hashCode() : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            if (nodeCnt.decrementAndGet() < 1) { //will throw exception on backup node only
+                throw new IOException("Class can not be unmarshalled");
+            }
+        }
+    }
+
+    /**
+     * Test key 2.
+     * Unmarshalling always failed.
+     */
+    public static class TestKeyAlwaysFailed extends TestKey {
+        /** Test key 2 */
+        public TestKeyAlwaysFailed(String field) {
+            super(field);
+        }
+
+        /** Test key 2 */
+        public TestKeyAlwaysFailed() {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            nodeCnt.decrementAndGet();
+            throw new IOException("Class can not be unmarshalled"); //will throw exception on primary node
+        }
+
+    }
+
+    /**
+     * Tests that correct response will be sent to client node in case of unmarshalling failed.
+     */
+    public void testResponseMessageOnUnmarshallingFailed() {
+
+        nodeCnt.set(1);
+
+        try {
+            jcache(0).put(new TestKeyAlwaysFailed("1"), "");
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert nodeCnt.get() == 0;//put request should not go to backup node in case failed at primary.
+
+        try {
+            assert jcache(0).get(new TestKeyAlwaysFailed("1")) == null;
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert grid(0).cachex().entrySet().size() == 0;
+
+        nodeCnt.set(2); //put request will be unmarshalled twice (at primary and at backup node).
+
+        try {
+            jcache(0).put(new TestKey("1"), "");//put will fail at backup node.
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert nodeCnt.get() == 0;//put request should go to primary and backup node.
+
+        // Need to have to exception while unmarshalling getKeyResponse.
+        nodeCnt.set(3); //get response will me unmarshalled twice (request at primary node and response at client).
+
+        assert jcache(0).get(new TestKey("1")) == null;
+
+        assert grid(0).cachex().entrySet().size() == 0;
+    }
+}


[08/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: 300cc75e8cb6546e1537081583e55e7fae165893
Parents: cdbaaba
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 23 16:03:50 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 23 16:03:50 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheP2pUnmarshallingErrorTest.java   | 76 +++++++++-----------
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 16 ++---
 2 files changed, 40 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/300cc75e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 7edbff2..1f622bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -100,11 +100,13 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
         /** {@inheritDoc} */
         @Override public void writeExternal(ObjectOutput out) throws IOException {
-
+            out.writeObject(field);
         }
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            field = (String)in.readObject();
+
             if (readCnt.decrementAndGet() <= 0) { //will throw exception on backup node only
                 throw new IOException("Class can not be unmarshalled");
             }
@@ -112,74 +114,60 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     }
 
     /**
-     * Test key 2. Unmarshalling always failed.
+     * Sends put atomically and handles fail.
      */
-    public static class TestKeyAlwaysFailed extends TestKey {
-        /** Test key 2. */
-        public TestKeyAlwaysFailed(String field) {
-            super(field);
-        }
+    protected void failAtomicPut() {
+        try {
+            jcache(0).put(new TestKey("1"), "");
 
-        /** Test key 2. */
-        public TestKeyAlwaysFailed() {
+            assert false : "p2p marshalling failed, but error response was not sent";
         }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            readCnt.decrementAndGet();
-            throw new IOException("Class can not be unmarshalled"); //will throw exception on primary node
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
         }
 
+        assert readCnt.get() == 0; //ensure we have read count as expected.
     }
 
     /**
-     * Tests that correct response will be sent to client node in case of unmarshalling failed.
+     * Sends get atomically and handles fail.
      */
-    public void testResponseMessageOnUnmarshallingFailed() {
-        //Checking failed unmarshalling on primary node.
-        readCnt.set(1);
-
+    protected void failAtomicGet() {
         try {
-            jcache(0).put(new TestKeyAlwaysFailed("1"), ""); //put will fail at primary node.
+            jcache(0).get(new TestKey("1"));
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
         catch (CacheException e) {
             assert X.hasCause(e, IOException.class);
         }
+    }
 
-        assert readCnt.get() == 0; //put request should not be handled by backup node in case failed at primary.
-
-        try {
-            assert jcache(0).get(new TestKeyAlwaysFailed("1")) == null;
+    /**
+     * Tests that correct response will be sent to client node in case of unmarshalling failed.
+     */
+    public void testResponseMessageOnUnmarshallingFailed() {
+        //GridNearAtomicUpdateRequest unmarshalling failed test
+        readCnt.set(1);
 
-            assert false : "p2p marshalling failed, but error response was not sent";
-        }
-        catch (CacheException e) {
-            assert X.hasCause(e, IOException.class);
-        }
+        failAtomicPut();
 
-        assert grid(0).cachex().entrySet().size() == 0;
+        //GridNearGetRequest unmarshalling failed test
+        readCnt.set(1);
 
-        //Checking failed unmarshalling on backup node.
-        readCnt.set(2); //put request will be unmarshalled twice (at primary and at backup node).
+        failAtomicGet();
 
-        try {
-            jcache(0).put(new TestKey("1"), ""); //put will fail at backup node only.
+        readCnt.set(100);
 
-            assert false : "p2p marshalling failed, but error response was not sent";
-        }
-        catch (CacheException e) {
-            assert X.hasCause(e, IOException.class);
-        }
+        assert jcache(0).get(new TestKey("1")) == null;
 
-        assert readCnt.get() == 0; //put request should be handled by primary and backup node.
+        readCnt.set(2);
 
-        // Need to have no exception while unmarshalling getKeyResponse.
-        readCnt.set(3); //get response will me unmarshalled twice (request at primary node and response at client).
+        //GridDhtAtomicUpdateRequest unmarshalling failed test
+        failAtomicPut();
 
-        assert jcache(0).get(new TestKey("1")) == null;
+        readCnt.set(100);
 
-        assert grid(0).cachex().entrySet().size() == 0;
+        assert jcache(0).get(new TestKey("1")) != null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/300cc75e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
index cb6d444..ed3749f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
@@ -50,7 +50,7 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
             assert X.hasCause(e, IOException.class);
         }
 
-        assert readCnt.get() == 0; //ensure we have read counts as expected.
+        assert readCnt.get() == 0; //ensure we have read count as expected.
     }
 
     /**
@@ -63,11 +63,11 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
-        catch (IgniteException e) {
-            assert X.hasCause(e, IOException.class);
-        }
+//        catch (IgniteException e) {
+//            assert X.hasCause(e, IOException.class);
+//        }
 
-        assert readCnt.get() == 0; //ensure we have read counts as expected.
+        assert readCnt.get() == 0; //ensure we have read count as expected.
     }
 
     /**
@@ -84,9 +84,9 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
 //
 //        failOptimistic();
 
-//        readCnt.set(1);
-//
-//        failPessimictic();
+        readCnt.set(100);
+
+        failPessimictic();
 
     }
 }


[10/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java


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

Branch: refs/heads/ignite-695
Commit: 9e4bc1044c7b6b3cc2bcbad4cae9bbebd33bcc14
Parents: d24064a 2dfc187
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 24 19:14:24 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 24 19:14:24 2015 +0300

----------------------------------------------------------------------
 .../aop/aspectj/GridifyAspectJAspect.java       |    2 +-
 .../aspectj/GridifySetToSetAspectJAspect.java   |    2 +-
 .../aspectj/GridifySetToValueAspectJAspect.java |    2 +-
 .../aop/spring/GridifySetToSetSpringAspect.java |    2 +-
 .../spring/GridifySetToValueSpringAspect.java   |    2 +-
 .../gridify/aop/spring/GridifySpringAspect.java |    2 +-
 .../cache/CacheServerNotFoundException.java     |   12 +-
 .../apache/ignite/cache/CachingProvider.java    |    3 +
 .../ignite/events/CacheQueryExecutedEvent.java  |    3 +-
 .../ignite/events/CacheQueryReadEvent.java      |    3 +-
 .../apache/ignite/internal/IgniteKernal.java    |   15 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    6 +
 .../ClusterTopologyServerNotFoundException.java |   12 +-
 .../deployment/GridDeploymentClassLoader.java   |    2 +-
 .../discovery/GridDiscoveryManager.java         |    3 +
 .../eventstorage/GridEventStorageManager.java   |    6 +
 .../affinity/GridAffinityAssignmentCache.java   |    4 +-
 .../cache/CacheStoreBalancingWrapper.java       |    6 +
 .../processors/cache/GridCacheAdapter.java      |    9 +-
 .../processors/cache/GridCacheEntryEx.java      |    4 +
 .../processors/cache/GridCacheIoManager.java    |    6 +
 .../processors/cache/GridCacheMapEntry.java     |   20 +-
 .../GridCachePartitionExchangeManager.java      |    2 +-
 .../processors/cache/GridCacheUtils.java        |   53 +-
 .../GridDistributedCacheAdapter.java            |   20 +-
 .../distributed/GridDistributedLockRequest.java |   99 +-
 .../GridDistributedTxRemoteAdapter.java         |    3 +
 .../distributed/dht/GridDhtCacheAdapter.java    |    4 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    9 +-
 .../distributed/dht/GridDhtLockFuture.java      |   20 +-
 .../distributed/dht/GridDhtLockRequest.java     |    7 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   21 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |    3 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   26 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    6 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |    7 +-
 .../dht/GridPartitionedGetFuture.java           |    7 +
 .../dht/atomic/GridDhtAtomicCache.java          |   41 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   38 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   49 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   56 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   18 +-
 .../GridDhtPartitionsExchangeFuture.java        |    3 +
 .../distributed/near/GridNearAtomicCache.java   |    5 +-
 .../distributed/near/GridNearCacheAdapter.java  |   10 +-
 .../distributed/near/GridNearCacheEntry.java    |   10 +-
 .../distributed/near/GridNearGetFuture.java     |   17 +-
 .../distributed/near/GridNearLockFuture.java    |   15 +-
 .../distributed/near/GridNearLockRequest.java   |    7 +-
 .../near/GridNearTransactionalCache.java        |   43 +-
 .../near/GridNearTxFinishFuture.java            |    3 +
 .../cache/distributed/near/GridNearTxLocal.java |    7 +-
 .../distributed/near/GridNearTxRemote.java      |    7 +-
 .../local/atomic/GridLocalAtomicCache.java      |   59 +-
 .../query/GridCacheDistributedQueryManager.java |    3 +
 .../cache/query/GridCacheLocalQueryFuture.java  |    3 +
 .../query/GridCacheQueryFutureAdapter.java      |    3 +
 .../cache/query/GridCacheQueryManager.java      |   20 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |    3 +
 .../cache/transactions/IgniteTxAdapter.java     |    7 +-
 .../cache/transactions/IgniteTxEntry.java       |   74 +-
 .../cache/transactions/IgniteTxHandler.java     |   27 +-
 .../transactions/IgniteTxLocalAdapter.java      |  106 +-
 .../cache/transactions/IgniteTxLocalEx.java     |    4 +-
 .../closure/GridClosureProcessor.java           |   16 +-
 .../datastreamer/DataStreamProcessor.java       |   28 +-
 .../datastreamer/DataStreamerImpl.java          |    3 +-
 .../datastreamer/DataStreamerRequest.java       |   38 +-
 .../processors/igfs/IgfsFileWorkerBatch.java    |    3 +
 .../internal/processors/igfs/IgfsThread.java    |    8 +-
 .../internal/processors/job/GridJobWorker.java  |    9 +
 .../portable/GridPortableInputStream.java       |   26 -
 .../processors/query/GridQueryProcessor.java    |    3 +
 .../processors/rest/GridRestProcessor.java      |    3 +
 .../service/GridServiceProcessor.java           |    9 +
 .../processors/task/GridTaskWorker.java         |   12 +
 .../timeout/GridTimeoutProcessor.java           |    3 +
 .../ignite/internal/util/IgniteUtils.java       |   22 +-
 .../util/ipc/loopback/IpcServerTcpEndpoint.java |    2 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    2 +-
 .../ignite/internal/util/lang/GridFunc.java     |   12 +-
 .../ignite/internal/util/nio/GridNioServer.java |    6 +
 .../apache/ignite/internal/util/typedef/X.java  |    2 +-
 .../ignite/internal/util/worker/GridWorker.java |    3 +
 .../visor/misc/VisorResolveHostNameTask.java    |    2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    6 +-
 .../ignite/messaging/MessagingListenActor.java  |    3 +
 .../org/apache/ignite/spi/IgniteSpiThread.java  |    3 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    3 +
 .../startup/cmdline/CommandLineStartup.java     |    5 +-
 .../startup/cmdline/CommandLineTransformer.java |    3 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |  514 ++++++-
 .../processors/cache/GridCacheTestEntryEx.java  |    2 +
 .../cache/IgniteExcangeFutureHistoryTest.java   |   74 -
 .../cache/IgniteExchangeFutureHistoryTest.java  |   77 ++
 .../CacheNoValueClassOnServerNodeTest.java      |  129 ++
 .../DataStreamerMultiThreadedSelfTest.java      |  101 ++
 .../ignite/testsuites/IgniteCacheTestSuite.java |    7 +-
 .../CacheNoValueClassOnServerTestClient.java    |   88 ++
 .../apache/ignite/tests/p2p/cache/Person.java   |   42 +
 .../CacheConfigurationP2PTestClient.java        |    1 -
 .../processors/hadoop/HadoopDefaultJobInfo.java |    3 +
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    3 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    5 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |    3 +
 .../external/HadoopExternalTaskExecutor.java    |    3 +
 .../processors/hadoop/v2/HadoopV2Job.java       |   11 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |    9 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |    3 +
 .../GridCacheAbstractFieldsQuerySelfTest.java   | 1284 ------------------
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  235 +++-
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |   34 -
 ...GridCachePartitionedFieldsQuerySelfTest.java |  115 --
 ...rtitionedFieldsQueryP2PDisabledSelfTest.java |   34 -
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |   34 +
 ...eReplicatedFieldsQueryP2PEnableSelfTest.java |   34 -
 .../GridCacheReplicatedFieldsQuerySelfTest.java |  157 ---
 ...eplicatedFieldsQueryP2PDisabledSelfTest.java |   34 -
 ...ReplicatedFieldsQueryP2PEnabledSelfTest.java |   34 +
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../http/jetty/GridJettyRestHandler.java        |    3 +
 .../ignite/schema/ui/SchemaImportApp.java       |    8 +-
 .../uri/GridUriDeploymentClassLoader.java       |    4 +-
 123 files changed, 2234 insertions(+), 2141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e4bc104/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index affdaf0,b8668e6..1fe1f50
--- 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
@@@ -288,7 -282,16 +288,10 @@@ public class GridCacheIoManager extend
              }
          }
          catch (Throwable e) {
 -            if (X.hasCause(e, ClassNotFoundException.class))
 -                U.error(log, "Failed to process message (note that distributed services " +
 -                    "do not support peer class loading, if you deploy distributed service " +
 -                    "you should have all required classes in CLASSPATH on all nodes in topology) " +
 -                    "[senderId=" + nodeId + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
 -            else
 -                U.error(log, "Failed to process message [senderId=" + nodeId + ']', e);
 +            U.error(log, "Failed to process message [senderId=" + nodeId + ", messageType=" + cacheMsg.getClass() + ']', e);
+ 
+             if (e instanceof Error)
+                 throw (Error)e;
          }
          finally {
              if (depEnabled)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e4bc104/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------


[18/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: debdc2beb84ca2a525fddad18094afc12a39be26
Parents: cf6e435 bd3a572
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 18:45:21 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 18:45:21 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |  44 ++++
 .../org/apache/ignite/internal/IgnitionEx.java  | 165 +++++++++++++-
 .../util/spring/IgniteSpringHelper.java         |  54 ++++-
 .../util/spring/IgniteSpringHelperImpl.java     | 217 +++++++++++++++----
 .../IgniteStartFromStreamConfigurationTest.java |  50 +++++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 6 files changed, 487 insertions(+), 45 deletions(-)
----------------------------------------------------------------------



[16/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: 45c6e09b8d4c79683336687724d441c1cfad659e
Parents: 7814b53
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 18:19:14 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 18:19:14 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  2 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  2 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java | 15 ++++++-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  2 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  1 +
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java | 41 ++++++++++++++++++++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |  3 ++
 8 files changed, 62 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 2a3f08a..5aa0ac8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -27,7 +27,7 @@ import java.io.*;
 import java.util.concurrent.atomic.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTest {
     /** Allows to change behavior of readExternal method. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index 0b9226f..453812b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.configuration.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
index 95d38e1..49f58f9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -18,9 +18,13 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import javax.cache.*;
+import java.io.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
     /** {@inheritDoc} */
@@ -64,6 +68,13 @@ public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2
 
         readCnt.set(1);
 
-        jcache(3).get(new TestKey(String.valueOf(key)));
+        try {
+            jcache(3).get(new TestKey(String.valueOf(key)));
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
index ca48507..a4e2753 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
@@ -27,7 +27,7 @@ import javax.cache.*;
 import java.io.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/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 5d53129..c4a5fd5 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
@@ -46,6 +46,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTest(IgniteStartUpTestSuite.suite());
         suite.addTest(IgniteExternalizableSelfTestSuite.suite());
         suite.addTest(IgniteP2PSelfTestSuite.suite());
+        suite.addTest(IgniteCacheP2pUnmarshallingErrorTestSuit.suite());
 
         suite.addTest(new TestSuite(GridSelfTest.class));
         suite.addTest(new TestSuite(GridProjectionSelfTest.class));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java
new file mode 100644
index 0000000..9b40280
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java
@@ -0,0 +1,41 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Checks behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingErrorTestSuit extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("P2p Unmarshalling Test Suite");
+
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingNearErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingRebalanceErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingTxErrorTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index afb96be..c5f9632 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.configuration.*;
 import javax.cache.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index ce05980..69d7548 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -111,6 +111,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
 
         suite.addTestSuite(GridCacheQueryMetricsSelfTest.class);
 
+        //Unmarshallig query test.
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
+
         return suite;
     }
 }


[34/36] incubator-ignite git commit: # Minor tests refactoring.

Posted by sb...@apache.org.
# Minor tests refactoring.


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

Branch: refs/heads/ignite-695
Commit: 5f8f6f46dbba461e43dc23a292a2b0ff764f19dd
Parents: 57ab09e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue May 5 12:10:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue May 5 12:10:21 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheSwapReloadSelfTest.java      | 20 ++++++++++++++------
 .../cache/IgniteCachePeekModesAbstractTest.java | 15 ++++++++++++---
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |  4 ++--
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  | 11 ++++++++++-
 .../cache/GridCacheOffHeapSelfTest.java         | 11 ++++++++++-
 5 files changed, 48 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
index 78f01f0..7d4eefc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -35,7 +34,6 @@ import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.*;
-import javax.cache.configuration.*;
 import java.util.*;
 import java.util.concurrent.*;
 
@@ -51,6 +49,18 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
+    /**
+     * Creates swap space spi.
+     * @return The swap spi.
+     */
+    protected SwapSpaceSpi spi() {
+        FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
+
+        swap.setWriteBufferSize(1);
+
+        return swap;
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -62,11 +72,9 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(disco);
 
-        FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
-
-        swap.setWriteBufferSize(1);
+        SwapSpaceSpi spi = spi();
 
-        cfg.setSwapSpaceSpi(swap);
+        cfg.setSwapSpaceSpi(spi);
 
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
index c2eb56f..7cd8414 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 
 import javax.cache.*;
@@ -54,11 +55,19 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         return cfg;
     }
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected NearCacheConfiguration nearConfiguration() {
         return null;
@@ -231,7 +240,7 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
             for (Integer key : keys)
                 cache0.put(key, val);
 
-            FileSwapSpaceSpi swap = (FileSwapSpaceSpi)ignite(nodeIdx).configuration().getSwapSpaceSpi();
+            SwapSpaceSpi swap = ignite(nodeIdx).configuration().getSwapSpaceSpi();
 
             Set<Integer> swapKeys = new HashSet<>();
 
@@ -614,7 +623,7 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
      * @return Tuple with primary and backup keys.
      */
     private T2<List<Integer>, List<Integer>> swapKeys(int nodeIdx) {
-        FileSwapSpaceSpi swap = (FileSwapSpaceSpi)ignite(nodeIdx).configuration().getSwapSpaceSpi();
+        SwapSpaceSpi swap = ignite(nodeIdx).configuration().getSwapSpaceSpi();
 
         IgniteSpiCloseableIterator<Integer> it = swap.keyIterator(SPACE_NAME, null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
index abd80e1..d21fb13 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
@@ -49,7 +49,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
     private static final long MAX_ENTRIES = 9000000;
 
     /** Test duration. */
-    private static final long DURATION = 10 * 60 * 1000;
+    private static final long DURATION = 2 * 60 * 1000;
 
     /** Swap context. */
     private final SwapContext swapCtx = new SwapContext();
@@ -73,7 +73,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
     /**
      * @return An SPI instance to test.
      */
-    private SwapSpaceSpi spi() {
+    protected SwapSpaceSpi spi() {
         FileSwapSpaceSpi spi = new FileSwapSpaceSpi();
 
 //        spi.setConcurrencyLevel(N_THREADS);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 226f0fc..0c55551 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -30,6 +30,7 @@ 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.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -79,6 +80,14 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
     /** */
     private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -91,7 +100,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
index 460a64c..2d6855e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.marshaller.optimized.*;
 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.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -65,6 +66,14 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
     /** PeerClassLoadingLocalClassPathExclude enable. */
     private boolean excluded;
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -77,7 +86,7 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 


[20/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: 1f35f3aae55fff2b6ab634f5e12fc0e52f276e58
Parents: eb20e01
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 28 21:09:57 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 28 21:09:57 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtForceKeysResponse.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f35f3aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 1d49b34..5c6e24f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -43,7 +43,6 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
     private IgniteUuid miniId;
 
     /** Error. */
-    @GridDirectTransient
     private volatile IgniteCheckedException err;
 
     /** Missed (not found) keys. */


[09/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: d24064a124558014300b945432942abc2e4998df
Parents: 300cc75
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 23 19:11:36 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 23 19:11:36 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 41 +++++++++++++++---
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  3 ++
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 45 ++++++++++++++++++--
 3 files changed, 78 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d24064a1/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 b584b17..affdaf0 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
@@ -315,7 +315,10 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             case 34:{
                 GridDhtTxPrepareRequest req = (GridDhtTxPrepareRequest)msg;
 
-                GridDhtTxPrepareResponse res = new GridDhtTxPrepareResponse(req.version(), req.futureId(), req.miniId());
+                GridDhtTxPrepareResponse res = new GridDhtTxPrepareResponse(
+                    req.version(),
+                    req.futureId(),
+                    req.miniId());
 
                 res.error(req.classError());
 
@@ -327,7 +330,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             case 38: {
                 GridDhtAtomicUpdateRequest req = (GridDhtAtomicUpdateRequest)msg;
 
-                GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(ctx.cacheId(), req.futureVersion());
+                GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(
+                    ctx.cacheId(),
+                    req.futureVersion());
 
                 res.onError(req.classError());
 
@@ -339,7 +344,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             case 40: {
                 GridNearAtomicUpdateRequest req = (GridNearAtomicUpdateRequest)msg;
 
-                GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
+                GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(
+                    ctx.cacheId(),
                     nodeId,
                     req.futureVersion());
 
@@ -353,7 +359,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             case 49: {
                 GridNearGetRequest req = (GridNearGetRequest)msg;
 
-                GridNearGetResponse res = new GridNearGetResponse(ctx.cacheId(),
+                GridNearGetResponse res = new GridNearGetResponse(
+                    ctx.cacheId(),
                     req.futureId(),
                     req.miniId(),
                     req.version());
@@ -365,11 +372,32 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
+            case 51: {
+                GridNearLockRequest req = (GridNearLockRequest)msg;
+
+                GridNearLockResponse res = new GridNearLockResponse(
+                    ctx.cacheId(),
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    false,
+                    0,
+                    req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
             case 55: {
                 GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg;
 
-                GridNearTxPrepareResponse res = new GridNearTxPrepareResponse(req.version(), req.futureId(),
-                    req.miniId(), req.version(), null, null, null);
+                GridNearTxPrepareResponse res = new GridNearTxPrepareResponse(
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    req.version(),
+                    null, null, null);
 
                 res.error(req.classError());
 
@@ -378,7 +406,6 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
-
             default:
                 throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message="
                     + msg + "]");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d24064a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 1f622bd..a50f07c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -32,6 +32,9 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /** Allows to change behavior of readExternal method. */
     protected static AtomicInteger readCnt = new AtomicInteger();
 
+    /** iterable key */
+    protected static int key = 0;
+
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d24064a1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
index ed3749f..1a42407 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.transactions.*;
 
@@ -34,13 +35,17 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
         return CacheAtomicityMode.TRANSACTIONAL;
     }
 
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
     /**
      * Sends put with optimistic lock and handles fail.
      */
     protected void failOptimistic() {
         try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
 
-            jcache(0).put(new TestKey("1"), "");
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
 
             tx.commit();
 
@@ -59,7 +64,26 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
     protected void failPessimictic() {
         try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
 
-            jcache(0).put(new TestKey("1"), "");
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+//        catch (IgniteException e) {
+//            assert X.hasCause(e, IOException.class);
+//        }
+
+        assert readCnt.get() == 0; //ensure we have read count as expected.
+    }
+
+    /**
+     * Sends put with pessimistic lock and handles fail.
+     */
+    protected void failPessimicticOnCommit() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+            tx.commit();
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
@@ -84,9 +108,22 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
 //
 //        failOptimistic();
 
-        readCnt.set(100);
+//        //GridNearLockRequest unmarshalling failed test
+//        readCnt.set(2);
+//
+//        failPessimictic();
+
+        //? unmarshalling failed test
+        readCnt.set(1000);
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            TestKey tstKey = new TestKey(String.valueOf(++key));
+            jcache(0).put(tstKey, "");
+            jcache(0).lock(tstKey).lock();
+        }
+
+
 
-        failPessimictic();
 
     }
 }


[32/36] incubator-ignite git commit: # ignite-sprint-5 fixed test

Posted by sb...@apache.org.
# ignite-sprint-5 fixed test


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

Branch: refs/heads/ignite-695
Commit: da3d9e73d182b77d27902871c19817f439308b2b
Parents: 114a852
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 5 09:36:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 5 09:36:52 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridCacheDhtPreloadSelfTest.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da3d9e73/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index f04b87f..5b63c6a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -266,7 +266,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
                         ClusterNode node = evt.discoveryNode();
 
                         return evt.type() == EVT_CACHE_REBALANCE_STOPPED && node.id().equals(nodeId) &&
-                            evt.discoveryEventType() == EVT_NODE_LEFT;
+                            (evt.discoveryEventType() == EVT_NODE_LEFT || evt.discoveryEventType() == EVT_NODE_FAILED);
                     }
                 }, EVT_CACHE_REBALANCE_STOPPED));
 


[35/36] incubator-ignite git commit: # ignite-sprint-5: fix license headers

Posted by sb...@apache.org.
# ignite-sprint-5: fix license headers


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

Branch: refs/heads/ignite-695
Commit: 6f1d642167ebb4563a9c9c7b3429d84214b433cd
Parents: 5f8f6f4
Author: Artem Shutak <as...@gridgain.com>
Authored: Tue May 5 18:34:08 2015 +0300
Committer: Artem Shutak <as...@gridgain.com>
Committed: Tue May 5 18:34:08 2015 +0300

----------------------------------------------------------------------
 dev-tools/build.gradle                     | 15 +++++++--------
 dev-tools/src/main/groovy/jiraslurp.groovy | 15 +++++++--------
 2 files changed, 14 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f1d6421/dev-tools/build.gradle
----------------------------------------------------------------------
diff --git a/dev-tools/build.gradle b/dev-tools/build.gradle
index 30ae6b7..b760bc1 100644
--- a/dev-tools/build.gradle
+++ b/dev-tools/build.gradle
@@ -1,13 +1,12 @@
 /*
- * 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
+ * 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
+ *      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,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f1d6421/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
index 332686f..0a876fa 100644
--- a/dev-tools/src/main/groovy/jiraslurp.groovy
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -1,13 +1,12 @@
 /*
- * 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
+ * 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
+ *      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,


[07/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: cdbaaba4c94b7670aae3e174419ec5cf61d4030b
Parents: 0d982f7 75425c6
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 23 15:01:56 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 23 15:01:56 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheInterceptor.java   |   9 +-
 .../configuration/CacheConfiguration.java       |  30 ++-
 .../configuration/IgniteConfiguration.java      | 107 +++++++++++
 .../ignite/configuration/TopologyValidator.java |  35 ++++
 .../org/apache/ignite/internal/IgnitionEx.java  |   8 +-
 .../ignite/internal/MarshallerContextImpl.java  |   2 +-
 .../connection/GridClientNioTcpConnection.java  |   9 +-
 .../managers/communication/GridIoManager.java   |  33 +++-
 .../GridLifecycleAwareMessageFilter.java        |  35 ++++
 .../discovery/GridDiscoveryManager.java         |  18 +-
 .../cache/GridCacheAffinityManager.java         |  12 --
 .../GridCachePartitionExchangeManager.java      |   5 +-
 .../processors/cache/IgniteCacheProxy.java      |  15 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   2 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |   8 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   7 +
 .../colocated/GridDhtColocatedLockFuture.java   |  24 ++-
 .../preloader/GridDhtPartitionSupplyPool.java   |   2 +-
 .../GridDhtPartitionsExchangeFuture.java        |  17 ++
 .../distributed/near/GridNearAtomicCache.java   |   2 +-
 .../distributed/near/GridNearLockFuture.java    |  26 ++-
 .../near/GridNearTransactionalCache.java        |   2 +-
 .../near/GridNearTxPrepareFuture.java           |  20 ++
 .../transactions/IgniteTxLocalAdapter.java      |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   2 +-
 .../processors/job/GridJobProcessor.java        |   6 +
 .../processors/query/GridQueryIndexing.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../resources/META-INF/classnames.properties    |  41 ++---
 .../cache/GridCacheAbstractFullApiSelfTest.java |   4 +-
 .../cache/IgniteExcangeFutureHistoryTest.java   |  74 ++++++++
 ...gniteTopologyValidatorAbstractCacheTest.java | 183 +++++++++++++++++++
 ...iteTopologyValidatorAbstractTxCacheTest.java | 125 +++++++++++++
 ...ValidatorNearPartitionedAtomicCacheTest.java |  30 +++
 ...logyValidatorNearPartitionedTxCacheTest.java |  30 +++
 ...logyValidatorPartitionedAtomicCacheTest.java |  49 +++++
 ...TopologyValidatorPartitionedTxCacheTest.java |  30 +++
 ...ologyValidatorReplicatedAtomicCacheTest.java |  49 +++++
 ...eTopologyValidatorReplicatedTxCacheTest.java |  30 +++
 .../IgniteCachePutGetRestartAbstractTest.java   |   2 +-
 ...arDisabledFairAffinityPutGetRestartTest.java |  35 ++++
 ...acheSortedEvictionPolicyPerformanceTest.java | 135 ++++++++++++++
 .../GridCacheLocalIsolatedNodesSelfTest.java    | 102 +++++++++++
 .../testframework/junits/GridAbstractTest.java  |  15 ++
 .../IgniteCacheFailoverTestSuite.java           |   3 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +
 .../IgniteTopologyValidatorTestSuit.java        |  43 +++++
 .../processors/query/h2/IgniteH2Indexing.java   |   6 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  34 +++-
 .../h2/twostep/GridReduceQueryExecutor.java     |  34 +++-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   2 +-
 .../ignite/schema/generator/CodeGenerator.java  |  60 +++++-
 .../ignite/internal/GridFactorySelfTest.java    |   3 +-
 pom.xml                                         |  14 +-
 54 files changed, 1480 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cdbaaba4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------


[24/36] incubator-ignite git commit: Merge branch 'ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: 2f24ee34b3033fc9789b46dd1afdff90abe7cd92
Parents: d617759 81ce0e6
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Apr 30 09:56:40 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Apr 30 09:56:40 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  2 +-
 .../ignite/internal/GridUpdateNotifier.java     | 66 +++++++++++++---
 .../apache/ignite/internal/IgniteKernal.java    | 83 +++++++++-----------
 .../org/apache/ignite/internal/IgnitionEx.java  | 15 +---
 .../org/apache/ignite/spi/IgniteSpiAdapter.java | 19 +++--
 .../internal/GridUpdateNotifierSelfTest.java    | 13 ++-
 6 files changed, 121 insertions(+), 77 deletions(-)
----------------------------------------------------------------------



[14/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: a55477970a2fdae5e93bcace708c98892937b97b
Parents: a8a9d10
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 17:42:03 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 17:42:03 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java | 6 +++---
 .../cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java   | 2 +-
 .../cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java       | 2 +-
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5547797/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index b358b15..2a3f08a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -158,7 +158,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         failAtomicPut(++key);
 
         //Check that cache is empty.
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         assert jcache(0).get(new TestKey(String.valueOf(key))) == null;
 
@@ -168,7 +168,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         failAtomicPut(++key);
 
         //Check that cache is not empty.
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         assert jcache(0).get(new TestKey(String.valueOf(key))) != null;
 
@@ -178,7 +178,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         failAtomicGet(++key);
 
         //GridNearGetResponse unmarshalling failed test
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         jcache(0).put(new TestKey(String.valueOf(++key)), "");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5547797/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
index f247a00..95d38e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -31,7 +31,7 @@ public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() throws Exception {
         //GridDhtPartitionSupplyMessage unmarshalling failed test
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         for (int i = 0; i <= 20; i++) {
             jcache(0).put(new TestKey(String.valueOf(++key)), "");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5547797/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index 265490c..afb96be 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -37,7 +37,7 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
 
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() {
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         TestKey val = new TestKey(String.valueOf(++key));
 


[33/36] incubator-ignite git commit: Merge branch 'ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: 57ab09ee168d70cbd65266944d263681c51a4f3c
Parents: da3d9e7 54f9492
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue May 5 12:09:09 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue May 5 12:09:09 2015 +0300

----------------------------------------------------------------------
 .../streaming/wordcount/CacheConfig.java        |   5 -
 .../config/grid-client-config.properties        |  50 +-
 .../ClientPropertiesConfigurationSelfTest.java  |  12 +-
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../org/apache/ignite/IgniteJdbcDriver.java     |  81 ++-
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../client/GridClientConfiguration.java         |   2 +-
 .../processors/cache/GridCacheAdapter.java      | 503 +++++++++----------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   4 +-
 pom.xml                                         |  39 +-
 11 files changed, 389 insertions(+), 435 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57ab09ee/pom.xml
----------------------------------------------------------------------


[04/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: 03c54d8907a4d77262e659dcae35ac509ad9c480
Parents: a47974c 6304fab
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 21 16:33:42 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 21 16:33:42 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheEvictionManager.java         | 11 +++++-
 .../dht/GridDhtAffinityAssignmentResponse.java  | 21 +++++++++++-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  2 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |  6 ++--
 .../dht/GridCacheDhtPreloadSelfTest.java        |  4 +--
 ...ledFairAffinityMultiNodeFullApiSelfTest.java | 36 ++++++++++++++++++++
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java | 36 ++++++++++++++++++++
 ...ledFairAffinityMultiNodeFullApiSelfTest.java | 36 ++++++++++++++++++++
 ...derFairAffinityMultiNodeFullApiSelfTest.java | 36 ++++++++++++++++++++
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java | 35 +++++++++++++++++++
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |  1 +
 ...nedFairAffinityMultiNodeFullApiSelfTest.java |  3 +-
 .../IgniteCacheFullApiSelfTestSuite.java        | 11 ++++--
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |  1 +
 15 files changed, 228 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[30/36] incubator-ignite git commit: Release procedure changed

Posted by sb...@apache.org.
Release procedure changed


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

Branch: refs/heads/ignite-695
Commit: 8c2e03e43133d3a546e254fe827a2c4ffc79966a
Parents: 445c0a6
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:54:50 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:54:50 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt | 32 ++++++++++++--------------------
 1 file changed, 12 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c2e03e4/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 7618e6c..1562dc4 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -40,35 +40,27 @@ Specify gpg profile at settings.xml. It will be used to sign sources and artifac
    </properties>
 </profile>
 
-Configure access to people.apache.org/~<username>. It necessary to upload sources to site.
-See https://blogs.apache.org/infra/entry/committer_shell_access_to_people for details.
-Specify ignite-site server at settings.xml.
-
-<server>
-   <id>ignite-site</id>
-   <username>username</username>
-   <privateKey>/home/username.ppk</privateKey>
-   <passphrase>passphrase</passphrase>
-</server>
-
 Ensure you have RELEASE (not SNAPSHOT) version at Ignite poms.
 Maven release plugin release:prepare goal can be used to make release tag.
 
-Deploy Ignite release to maven repository and site:
+Deploy Ignite release candidate to maven repository and dev-svn:
 
-mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -Dignite.site=scp://people.apache.org:/home/<username>/public_html -B
+   mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -B
 
-You might need to allow connection to people.apache.org. Just do following at console:
+   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern.
+   Sources package name is fixed. Sources package zip will be created automatically when apache-release profile used.
 
-ssh people.apache.org
+   In case you want to release both fabric and hadoop editions you have to build hadoop first, save /target/bin/*.zip,
+   make "mvn clean" and restore them before deploy step.
 
-and type "yes" + enter.
+   Nexus staging (repository.apache.org) should be closed when mvn deploy finished.
+   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create new folder with name equals to released
+   version at svn root. Copy target/site folder content to svn/ignite/<version> folder and commit.
 
-In case you want to release both fabric and hadoop editions you have to build hadoop first, save /target/bin/*.zip, make "mvn clean" and
-restore them before deploy step.
+Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>.
 
-Start vote based on people.apache.org/~<username>/ignite-version.
-Release nexus staging when version accepted.
+Release nexus staging and copy binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>
+to https://dist.apache.org/repos/dist/release/incubator/ignite/<version> when version accepted.
 
 
 JCache TCK compliance


[19/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: eb20e01a62dcbbd24b12f818b24cfa1af6261576
Parents: debdc2b acc731c
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 28 20:32:52 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 28 20:32:52 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |   2 +-
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../ignite/internal/GridJobContextImpl.java     |   7 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |   6 +-
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 +++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |  10 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../distributed/near/GridNearLockRequest.java   |  54 +-
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 +-
 .../cache/transactions/IgniteTxManager.java     | 117 +++-
 .../ignite/internal/util/GridJavaProcess.java   |   4 +
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../ignite/internal/visor/cache/VisorCache.java |  92 ++--
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../visor/node/VisorBasicConfiguration.java     |  11 +
 .../visor/node/VisorNodeDataCollectorJob.java   |   2 +-
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |   2 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 .../internal/GridDiscoveryEventSelfTest.java    |  30 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 +++++
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   5 +-
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  34 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 438 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 286 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 131 files changed, 3445 insertions(+), 2923 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eb20e01a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------


[26/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: d0798504652d09b583c7e9436dd619461aea1b05
Parents: b4c424f 7121619
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 16:12:38 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 16:12:38 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 168 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 pom.xml                                         |  51 ++++++
 4 files changed, 166 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


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


[13/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: a8a9d1089d3bf3d6a4adb56a71aede309fa42977
Parents: 163be30
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 17:41:10 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 17:41:10 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  72 +++++++++++-
 .../atomic/GridNearAtomicUpdateResponse.java    |   2 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   6 +
 .../dht/preloader/GridDhtForceKeysResponse.java |  19 ++++
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  41 ++++---
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 111 -------------------
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  56 ++++++++++
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  69 ++++++++++++
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 109 ++++++++++++++++++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  58 ++++++++++
 10 files changed, 410 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/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 c9af788..14d6f7c 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
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.processors.affinity.*;
 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.dht.preloader.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -301,7 +302,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
-    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx, GridIoPolicy plc) {
+    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx,
+        GridIoPolicy plc) {
         try {
             cctx.io().send(nodeId, res, plc);
         }
@@ -311,10 +313,24 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
-    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException{
+    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
+            case 14: {
+                GridCacheEvictionRequest req = (GridCacheEvictionRequest)msg;
+
+                GridCacheEvictionResponse res = new GridCacheEvictionResponse(
+                    ctx.cacheId(),
+                    req.futureId(),
+                    req.classError() != null
+                );
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
             case 30: {
                 GridDhtLockRequest req = (GridDhtLockRequest)msg;
 
@@ -330,7 +346,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
-            case 34:{
+            case 34: {
                 GridDhtTxPrepareRequest req = (GridDhtTxPrepareRequest)msg;
 
                 GridDhtTxPrepareResponse res = new GridDhtTxPrepareResponse(
@@ -367,13 +383,37 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     nodeId,
                     req.futureVersion());
 
-                res.onError(req.classError());
+                res.error(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
+            case 42: {
+                GridDhtForceKeysRequest req = (GridDhtForceKeysRequest)msg;
+
+                GridDhtForceKeysResponse res = new GridDhtForceKeysResponse(
+                    ctx.cacheId(),
+                    req.futureId(),
+                    req.miniId()
+                );
+
+                res.error(req.classError());
 
                 sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
             }
 
             break;
 
+            case 45: {
+                GridDhtPartitionSupplyMessage req = (GridDhtPartitionSupplyMessage)msg;
+
+                U.error(log, "Supply message cannot be unmarshalled.", req.classError());
+            }
+
+            break;
+
             case 49: {
                 GridNearGetRequest req = (GridNearGetRequest)msg;
 
@@ -390,6 +430,26 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
+            case 50: {
+                GridNearGetResponse res = (GridNearGetResponse)msg;
+
+                GridPartitionedGetFuture fut = (GridPartitionedGetFuture)ctx.mvcc().future(
+                    res.version(), res.futureId());
+
+                if (fut == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to find future for get response [sender=" + nodeId + ", res=" + res + ']');
+
+                    return;
+                }
+
+                res.error(res.classError());
+
+                fut.onResult(nodeId, res);
+            }
+
+            break;
+
             case 51: {
                 GridNearLockRequest req = (GridNearLockRequest)msg;
 
@@ -878,7 +938,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         catch (Error e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, NoClassDefFoundError.class,
                 UnsupportedClassVersionError.class))
-                    cacheMsg.onClassError(new IgniteCheckedException("Failed to load class during unmarshalling: " + e, e));
+                cacheMsg.onClassError(new IgniteCheckedException("Failed to load class during unmarshalling: " + e, e));
             else
                 throw e;
         }
@@ -907,7 +967,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
 
         /** {@inheritDoc} */
-        @SuppressWarnings( {"CatchGenericClass", "unchecked"})
+        @SuppressWarnings({"CatchGenericClass", "unchecked"})
         @Override public void onMessage(final UUID nodeId, Object msg) {
             if (log.isDebugEnabled())
                 log.debug("Received cache ordered message [nodeId=" + nodeId + ", msg=" + msg + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 773b847..330e43c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -140,7 +140,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * Sets update error.
      * @param err
      */
-    public void onError(IgniteCheckedException err){
+    public void error(IgniteCheckedException err){
         this.err = err;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 61aaa14..78966d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -451,6 +451,12 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
          * @param res Result callback.
          */
         void onResult(GridDhtForceKeysResponse res) {
+            if (res.error() != null) {
+                onDone(res.error());
+
+                return;
+            }
+
             Collection<KeyCacheObject> missedKeys = res.missedKeys();
 
             boolean remapMissed = false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 8919185..1d49b34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -42,6 +42,10 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
     /** Mini-future ID. */
     private IgniteUuid miniId;
 
+    /** Error. */
+    @GridDirectTransient
+    private volatile IgniteCheckedException err;
+
     /** Missed (not found) keys. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -73,6 +77,21 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
         this.miniId = miniId;
     }
 
+    /**
+     * Sets error.
+     * @param err
+     */
+    public void error(IgniteCheckedException err){
+        this.err = err;
+    }
+
+    /**
+     * @return Error, if any.
+     */
+    public IgniteCheckedException error() {
+        return err;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean allowForStartup() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 277f10d..b358b15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.annotations.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -82,6 +83,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         }
 
         /** Field. */
+        @QuerySqlField(index = true)
         private String field;
 
         /** {@inheritDoc} */
@@ -110,7 +112,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             field = (String)in.readObject();
 
-            if (readCnt.decrementAndGet() <= 0) { //will throw exception on backup node only
+            if (readCnt.decrementAndGet() <= 0) {
                 throw new IOException("Class can not be unmarshalled");
             }
         }
@@ -119,9 +121,9 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /**
      * Sends put atomically and handles fail.
      */
-    protected void failAtomicPut() {
+    protected void failAtomicPut(int k) {
         try {
-            jcache(0).put(new TestKey("1"), "");
+            jcache(0).put(new TestKey(String.valueOf(k)), "");
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
@@ -135,9 +137,9 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /**
      * Sends get atomically and handles fail.
      */
-    protected void failAtomicGet() {
+    protected void failAtomicGet(int k) {
         try {
-            jcache(0).get(new TestKey("1"));
+            jcache(0).get(new TestKey(String.valueOf(k)));
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
@@ -149,30 +151,39 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /**
      * Tests that correct response will be sent to client node in case of unmarshalling failed.
      */
-    public void testResponseMessageOnUnmarshallingFailed() {
+    public void testResponseMessageOnUnmarshallingFailed() throws Exception {
         //GridNearAtomicUpdateRequest unmarshalling failed test
         readCnt.set(1);
 
-        failAtomicPut();
-
-        //GridNearGetRequest unmarshalling failed test
-        readCnt.set(1);
-
-        failAtomicGet();
+        failAtomicPut(++key);
 
         //Check that cache is empty.
         readCnt.set(100);
 
-        assert jcache(0).get(new TestKey("1")) == null;
+        assert jcache(0).get(new TestKey(String.valueOf(key))) == null;
 
         //GridDhtAtomicUpdateRequest unmarshalling failed test
         readCnt.set(2);
 
-        failAtomicPut();
+        failAtomicPut(++key);
 
         //Check that cache is not empty.
         readCnt.set(100);
 
-        assert jcache(0).get(new TestKey("1")) != null;
+        assert jcache(0).get(new TestKey(String.valueOf(key))) != null;
+
+        //GridNearGetRequest unmarshalling failed test
+        readCnt.set(1);
+
+        failAtomicGet(++key);
+
+        //GridNearGetResponse unmarshalling failed test
+        readCnt.set(100);
+
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+        readCnt.set(2);
+
+        failAtomicGet(key);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
deleted file mode 100644
index 06176aa..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.transactions.*;
-
-import javax.cache.*;
-import java.io.*;
-
-/**
- * Check behavior on exception while unmarshalling key.
- */
-public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmarshallingErrorTest {
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        if (!gridName.endsWith("0"))
-            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
-
-        return cfg;
-    }
-
-    /**
-     * Sends put with optimistic lock and handles fail.
-     */
-    protected void failOptimistic() {
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-
-            jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
-            tx.commit();
-
-            assert false : "p2p marshalling failed, but error response was not sent";
-        }
-        catch (IgniteException e) {
-            assert X.hasCause(e, IOException.class);
-        }
-
-        assert readCnt.get() == 0; //ensure we have read count as expected.
-    }
-
-    /**
-     * Sends put with pessimistic lock and handles fail.
-     */
-    protected void failPessimictic() {
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-
-            jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
-            assert false : "p2p marshalling failed, but error response was not sent";
-        }
-        catch (CacheException e) {
-            assert X.hasCause(e, IOException.class);
-        }
-
-        assert readCnt.get() == 0; //ensure we have read count as expected.
-    }
-
-    /**
-     * Tests that correct response will be sent to client node in case of unmarshalling failed.
-     */
-    public void testResponseMessageOnUnmarshallingFailed() {
-        //GridNearTxPrepareRequest unmarshalling failed test
-        readCnt.set(2);
-
-        failOptimistic();
-
-        //GridDhtTxPrepareRequest unmarshalling failed test
-        readCnt.set(3);
-
-        failOptimistic();
-
-        //GridNearLockRequest unmarshalling failed test
-        readCnt.set(2);
-
-        failPessimictic();
-
-        //GridDhtLockRequest unmarshalling failed test
-        readCnt.set(3);
-
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-            jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
new file mode 100644
index 0000000..0b9226f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -0,0 +1,56 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.eviction.fifo.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.getCacheConfiguration()[0].setEvictMaxOverflowRatio(0);
+        cfg.getCacheConfiguration()[0].setEvictSynchronized(true);
+        cfg.getCacheConfiguration()[0].setEvictSynchronizedKeyBufferSize(1);
+        cfg.getCacheConfiguration()[0].setEvictionPolicy(new FifoEvictionPolicy(1));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() throws InterruptedException {
+        //GridCacheEvictionRequest unmarshalling failed test
+        readCnt.set(5);
+
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+        //Eviction request unmarshalling failed but ioManager does not hangs up.
+
+        Thread.sleep(1000); //todo: wait for eviction complete
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
new file mode 100644
index 0000000..f247a00
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -0,0 +1,69 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.affinity.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() throws Exception {
+        //GridDhtPartitionSupplyMessage unmarshalling failed test
+        readCnt.set(100);
+
+        for (int i = 0; i <= 20; i++) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+        }
+
+        readCnt.set(1);
+
+        startGrid(3);
+
+        //GridDhtPartitionSupplyMessage unmarshalling failed but ioManager does not hangs up.
+
+        Thread.sleep(1000);
+
+        //GridDhtForceKeysRequest unmarshalling failed test
+        stopGrid(3);
+
+        readCnt.set(Integer.MAX_VALUE);
+
+        for (int i = 0; i <= 1000; i++) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+        }
+
+        startGrid(3);
+
+        Affinity<Object> aff = affinity(grid(3).cache(null));
+
+        while (!aff.isPrimary(grid(3).localNode(), new TestKey(String.valueOf(key)))) {
+            --key;
+        }
+
+        readCnt.set(1);
+
+        jcache(3).get(new TestKey(String.valueOf(key)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
new file mode 100644
index 0000000..ca48507
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.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.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import java.io.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (!gridName.endsWith("0"))
+            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
+
+        return cfg;
+    }
+
+    /**
+     * Sends put with optimistic lock and handles fail.
+     */
+    protected void failOptimistic() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+            tx.commit();
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (IgniteException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert readCnt.get() == 0; //ensure we have read count as expected.
+    }
+
+    /**
+     * Sends put with pessimistic lock and handles fail.
+     */
+    protected void failPessimictic() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert readCnt.get() == 0; //ensure we have read count as expected.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() {
+        //GridNearTxPrepareRequest unmarshalling failed test
+        readCnt.set(2);
+
+        failOptimistic();
+
+        //GridDhtTxPrepareRequest unmarshalling failed test
+        readCnt.set(3);
+
+        failOptimistic();
+
+        //GridNearLockRequest unmarshalling failed test
+        readCnt.set(2);
+
+        failPessimictic();
+
+        //GridDhtLockRequest unmarshalling failed test
+        readCnt.set(3);
+
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
new file mode 100644
index 0000000..265490c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.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.cache;
+
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+
+import javax.cache.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.getCacheConfiguration()[0].setIndexedTypes(TestKey.class, String.class);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() {
+        readCnt.set(100);
+
+        TestKey val = new TestKey(String.valueOf(++key));
+
+        jcache(0).put(val, "");
+
+        //GridCacheQueryRequest unmarshalling failed test
+        readCnt.set(1);
+
+        try {
+            jcache(0).query(new SqlQuery<TestKey, String>(String.class, "field like '" + key + "'")).getAll();
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            // No-op
+        }
+    }
+}


[15/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: 7814b53ca63325d33d2340563f00b0c5a06355c2
Parents: a554779 5fb7948
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 17:47:15 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 17:47:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheInvokeResult.java     | 24 ++++++++++++---
 .../processors/cache/GridCacheAdapter.java      |  2 +-
 .../processors/cache/GridCacheMapEntry.java     |  4 +--
 .../processors/cache/GridCacheReturn.java       |  5 +--
 .../dht/atomic/GridDhtAtomicCache.java          |  3 +-
 .../local/atomic/GridLocalAtomicCache.java      |  6 ++--
 .../top/GridTopologyCommandHandler.java         |  3 +-
 .../DataStreamerMultiThreadedSelfTest.java      | 19 +++++++++---
 .../GridSwapSpaceSpiAbstractSelfTest.java       |  2 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    | 32 ++++++++++----------
 10 files changed, 63 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[29/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: 445c0a61544fc3d05cef77b0c0653b93859e7da7
Parents: e406987 44b00ea
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:53:36 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:53:36 2015 +0300

----------------------------------------------------------------------
 pom.xml | 46 +++++++++++++++++++++++-----------------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/445c0a61/pom.xml
----------------------------------------------------------------------


[05/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: 9a79e70a11eec1688968f33f99892b399b94254b
Parents: 03c54d8
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 21 17:02:23 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 21 17:02:23 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheIoManager.java     | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9a79e70a/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 0df824f..957f3c8 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
@@ -302,12 +302,12 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             ctx.io().send(nodeId, res, ctx.ioPolicy());
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId +
+            U.error(log, "Failed to send response to node (is node still alive?) [nodeId=" + nodeId +
                 ",res=" + res + ']', e);
         }
     }
 
-    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) {
+    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException{
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
@@ -322,6 +322,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
 
             break;
+
             case 40: {
                 GridNearAtomicUpdateRequest req = (GridNearAtomicUpdateRequest)msg;
 
@@ -335,6 +336,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
 
             break;
+
             case 49: {
                 GridNearGetRequest req = (GridNearGetRequest)msg;
 
@@ -349,6 +351,10 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             }
 
             break;
+
+            default:
+                throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message="
+                    + msg + "]");
         }
     }
 


[23/36] incubator-ignite git commit: "Version changed

Posted by sb...@apache.org.
"Version changed


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

Branch: refs/heads/ignite-695
Commit: d617759da292917df4b827abc8d50b6152c4190e
Parents: bba8c2f
Author: Ignite Teamcity <ig...@apache.org>
Authored: Wed Apr 29 19:23:41 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Wed Apr 29 19:23:41 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml              | 2 +-
 modules/aop/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/extdata/p2p/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/log4j/pom.xml         | 2 +-
 modules/rest-http/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/spring/pom.xml        | 2 +-
 modules/ssh/pom.xml           | 2 +-
 modules/tools/pom.xml         | 2 +-
 modules/urideploy/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 +-
 pom.xml                       | 2 +-
 31 files changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 9dda753..f38ff9d 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index ef44603..1b03acd 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 43f24bc..057f246 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index ca93673..19d6a25 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 8cb97d0..91ba978 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 9e03dab..7cce037 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 62612f8..cdf2cf2 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 0f8fdc0..6509c47 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.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index df69c93..921775f 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 8b2a019..1061aee 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 1ce3370..6a7610c 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 231808f..9b5bcfb 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index ed4dc07..fbd40ce 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 6ff6039..2949c34 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 8a3cec3..04ab11b 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 9287f12..7322da2 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index dff0a7e..42c8523 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 9097614..72bf1ff 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.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 590d3f7..b8840e3 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 2c09ed9..ee733ad 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 7c49cab..d420681 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.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 1f78f42..9acb521 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 1390a39..a0bcaaa 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 867e9be..521c7d0 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 625eeaa..674f22f 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 231a576..939966c 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index 07e27a0..7abd896 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.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 4e58ab8..04dda81 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.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index de50712..446d7a0 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 6bd65cc..108fa62 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d617759d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0e84a50..e1badec 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite</artifactId>
-    <version>1.0.3-SNAPSHOT</version>
+    <version>1.1.0-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[17/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: cf6e4352b3913c67123a8d2329069373d4b05075
Parents: 45c6e09
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 18:34:49 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 18:34:49 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheP2pUnmarshallingNearErrorTest.java          | 2 +-
 .../cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java         | 4 +---
 2 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6e4352/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index 453812b..732d12d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -44,7 +44,7 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() throws InterruptedException {
         //GridCacheEvictionRequest unmarshalling failed test
-        readCnt.set(5);
+        readCnt.set(5); //2 for each put
 
         jcache(0).put(new TestKey(String.valueOf(++key)), "");
         jcache(0).put(new TestKey(String.valueOf(++key)), "");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6e4352/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index c5f9632..b2095a8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -39,9 +39,7 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
     @Override public void testResponseMessageOnUnmarshallingFailed() {
         readCnt.set(Integer.MAX_VALUE);
 
-        TestKey val = new TestKey(String.valueOf(++key));
-
-        jcache(0).put(val, "");
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
 
         //GridCacheQueryRequest unmarshalling failed test
         readCnt.set(1);


[25/36] incubator-ignite git commit: Merge branch 'ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: b4c424f2465f696f8c19f44e23bc77980761f005
Parents: 2f24ee3 2a176e4
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Apr 30 15:21:28 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Apr 30 15:21:28 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 assembly/release-base.xml                       |   2 +
 assembly/release-schema-import.xml              |  50 ++++++
 .../managers/communication/GridIoManager.java   |   8 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 +++++++++++++++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 -------------------
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 modules/schema-import/pom.xml                   |   6 +-
 pom.xml                                         | 137 ++++++++++++++---
 9 files changed, 335 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4c424f2/modules/schema-import/pom.xml
----------------------------------------------------------------------

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


[28/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: e406987615bef30b759fabe466abdfdab6d70e65
Parents: 957c206 139aa27
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:50:33 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:50:33 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 ++++++++++---------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 .../discovery/tcp/TcpClientDiscoverySpi.java    |   4 -
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 -
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   8 +-
 .../tcp/TcpClientDiscoverySelfTest.java         |   8 +
 pom.xml                                         |  46 +-
 8 files changed, 384 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


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


[03/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: a47974c3444da2f1804ca1c0b80cd74f92cdf137
Parents: 7a8e075
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 21 16:33:13 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 21 16:33:13 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 160 ++++++++++++-------
 .../distributed/dht/GridDhtCacheAdapter.java    |  19 ---
 .../dht/atomic/GridDhtAtomicCache.java          |  31 +---
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  46 +++---
 4 files changed, 131 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a47974c3/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 56ee65e..0df824f 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
@@ -24,6 +24,8 @@ import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -32,7 +34,6 @@ import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
-import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -226,68 +227,66 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             unmarshall(nodeId, cacheMsg);
 
-            if (cacheMsg.allowForStartup())
-                processMessage(nodeId, cacheMsg, c);
+            if (cacheMsg.classError() != null)
+                processFailedMessage(nodeId, cacheMsg);
             else {
-                IgniteInternalFuture<?> startFut = startFuture(cacheMsg);
-
-                if (startFut.isDone())
+                if (cacheMsg.allowForStartup())
                     processMessage(nodeId, cacheMsg, c);
                 else {
-                    if (log.isDebugEnabled())
-                        log.debug("Waiting for start future to complete for message [nodeId=" + nodeId +
-                            ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
-
-                    // Don't hold this thread waiting for preloading to complete.
-                    startFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                        @Override public void apply(final IgniteInternalFuture<?> f) {
-                            cctx.kernalContext().closure().runLocalSafe(
-                                new GridPlainRunnable() {
-                                    @Override public void run() {
-                                        rw.readLock();
-
-                                        try {
-                                            if (stopping) {
-                                                if (log.isDebugEnabled())
-                                                    log.debug("Received cache communication message while stopping " +
-                                                        "(will ignore) [nodeId=" + nodeId + ", msg=" + cacheMsg + ']');
+                    IgniteInternalFuture<?> startFut = startFuture(cacheMsg);
 
-                                                return;
-                                            }
+                    if (startFut.isDone())
+                        processMessage(nodeId, cacheMsg, c);
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Waiting for start future to complete for message [nodeId=" + nodeId +
+                                ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
 
-                                            f.get();
+                        // Don't hold this thread waiting for preloading to complete.
+                        startFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                            @Override public void apply(final IgniteInternalFuture<?> f) {
+                                cctx.kernalContext().closure().runLocalSafe(
+                                    new GridPlainRunnable() {
+                                        @Override public void run() {
+                                            rw.readLock();
 
-                                            if (log.isDebugEnabled())
-                                                log.debug("Start future completed for message [nodeId=" + nodeId +
-                                                    ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
+                                            try {
+                                                if (stopping) {
+                                                    if (log.isDebugEnabled())
+                                                        log.debug("Received cache communication message while stopping " +
+                                                            "(will ignore) [nodeId=" + nodeId + ", msg=" + cacheMsg + ']');
 
-                                            processMessage(nodeId, cacheMsg, c);
-                                        }
-                                        catch (IgniteCheckedException e) {
-                                            // Log once.
-                                            if (startErr.compareAndSet(false, true))
-                                                U.error(log, "Failed to complete preload start future " +
-                                                    "(will ignore message) " +
-                                                    "[fut=" + f + ", nodeId=" + nodeId + ", msg=" + cacheMsg + ']', e);
-                                        }
-                                        finally {
-                                            rw.readUnlock();
+                                                    return;
+                                                }
+
+                                                f.get();
+
+                                                if (log.isDebugEnabled())
+                                                    log.debug("Start future completed for message [nodeId=" + nodeId +
+                                                        ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
+
+                                                processMessage(nodeId, cacheMsg, c);
+                                            }
+                                            catch (IgniteCheckedException e) {
+                                                // Log once.
+                                                if (startErr.compareAndSet(false, true))
+                                                    U.error(log, "Failed to complete preload start future " +
+                                                        "(will ignore message) " +
+                                                        "[fut=" + f + ", nodeId=" + nodeId + ", msg=" + cacheMsg + ']', e);
+                                            }
+                                            finally {
+                                                rw.readUnlock();
+                                            }
                                         }
                                     }
-                                }
-                            );
-                        }
-                    });
+                                );
+                            }
+                        });
+                    }
                 }
             }
         }
         catch (Throwable e) {
-//            if (X.hasCause(e, ClassNotFoundException.class))
-//                U.error(log, "Failed to process message (note that distributed services " +
-//                    "do not support peer class loading, if you deploy distributed service " +
-//                    "you should have all required classes in CLASSPATH on all nodes in topology) " +
-//                    "[senderId=" + nodeId + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
-//            else
             U.error(log, "Failed to process message [senderId=" + nodeId + ", messageType=" + cacheMsg.getClass() + ']', e);
         }
         finally {
@@ -298,6 +297,61 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
+    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheContext ctx) {
+        try {
+            ctx.io().send(nodeId, res, ctx.ioPolicy());
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId +
+                ",res=" + res + ']', e);
+        }
+    }
+
+    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) {
+        GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
+
+        switch (msg.directType()) {
+            case 38: {
+                GridDhtAtomicUpdateRequest req = (GridDhtAtomicUpdateRequest)msg;
+
+                GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(ctx.cacheId(), req.futureVersion());
+
+                res.onError(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, ctx);
+            }
+
+            break;
+            case 40: {
+                GridNearAtomicUpdateRequest req = (GridNearAtomicUpdateRequest)msg;
+
+                GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
+                    nodeId,
+                    req.futureVersion());
+
+                res.onError(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, ctx);
+            }
+
+            break;
+            case 49: {
+                GridNearGetRequest req = (GridNearGetRequest)msg;
+
+                GridNearGetResponse res = new GridNearGetResponse(ctx.cacheId(),
+                    req.futureId(),
+                    req.miniId(),
+                    req.version());
+
+                res.error(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, ctx);
+            }
+
+            break;
+        }
+    }
+
     /**
      * @param cacheMsg Cache message to get start future.
      * @return Preloader start future.
@@ -738,11 +792,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             cacheMsg.finishUnmarshal(cctx, cctx.deploy().globalLoader());
         }
         catch (IgniteCheckedException e) {
-//            if (cacheMsg.ignoreClassErrors() && X.hasCause(e, InvalidClassException.class,
-//                    ClassNotFoundException.class, NoClassDefFoundError.class, UnsupportedClassVersionError.class))
-                cacheMsg.onClassError(e);
-//            else
-//                throw e;
+            cacheMsg.onClassError(e);
         }
         catch (Error e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, NoClassDefFoundError.class,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a47974c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index c6ebe0a..d85bc75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -606,25 +606,6 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     protected void processNearGetRequest(final UUID nodeId, final GridNearGetRequest req) {
         assert ctx.affinityNode();
 
-        if (req.classError() != null) {
-            GridNearGetResponse res = new GridNearGetResponse(ctx.cacheId(),
-                req.futureId(),
-                req.miniId(),
-                req.version());
-
-            res.error(req.classError());
-
-            try {
-                ctx.io().send(nodeId, res, ctx.ioPolicy());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId +
-                    ",req=" + req + ", res=" + res + ']', e);
-            }
-
-            return;
-        }
-
         long ttl = req.accessTtl();
 
         final CacheExpiryPolicy expiryPlc = CacheExpiryPolicy.forAccess(ttl);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a47974c3/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 ec9e5a6..85f11b5 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
@@ -2329,18 +2329,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         req.nodeId(ctx.localNodeId());
 
-        if (req.classError() != null) {
-            GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(),
-                nodeId,
-                req.futureVersion());
-
-            res.onError(req.classError());
-
-            sendNearUpdateReply(nodeId, res);
-        }
-        else {
-            updateAllAsyncInternal(nodeId, req, updateReplyClos);
-        }
+        updateAllAsyncInternal(nodeId, req, updateReplyClos);
     }
 
     /**
@@ -2376,24 +2365,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         // Always send update reply.
         GridDhtAtomicUpdateResponse res = new GridDhtAtomicUpdateResponse(ctx.cacheId(), req.futureVersion());
 
-        if (req.classError() != null) {
-            res.onError(req.classError());
-
-            try {
-                ctx.io().send(nodeId, res, ctx.ioPolicy());
-            }
-            catch (ClusterTopologyCheckedException ignored) {
-                U.warn(log, "Failed to send DHT atomic update response to node because it left grid: " +
-                    req.nodeId());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send DHT atomic update response (did node leave grid?) [nodeId=" + nodeId +
-                    ", req=" + req + ']', e);
-            }
-
-            return;
-        }
-
         Boolean replicate = ctx.isDrEnabled();
 
         boolean intercept = req.forceTransformBackups() && ctx.config().getInterceptor() != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a47974c3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index d21c219..60f2226 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -26,11 +26,11 @@ import java.io.*;
 import java.util.concurrent.atomic.*;
 
 /**
- * Check behavior on exception while unmarshalling key
+ * Check behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTest {
-    /** Allows to change behavior of readExternal method */
-    private static AtomicInteger nodeCnt = new AtomicInteger();
+    /** Allows to change behavior of readExternal method. */
+    private static AtomicInteger readCnt = new AtomicInteger();
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -67,18 +67,18 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         return cfg;
     }
 
-    /** Test key 1 */
+    /** Test key 1. */
     public static class TestKey implements Externalizable {
-        /** Test key 1 */
+        /** Test key 1. */
         public TestKey(String field) {
             this.field = field;
         }
 
-        /** Test key 1 */
+        /** Test key 1. */
         public TestKey() {
         }
 
-        /** field */
+        /** field. */
         private String field;
 
         /** {@inheritDoc} */
@@ -105,29 +105,28 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            if (nodeCnt.decrementAndGet() < 1) { //will throw exception on backup node only
+            if (readCnt.decrementAndGet() <= 0) { //will throw exception on backup node only
                 throw new IOException("Class can not be unmarshalled");
             }
         }
     }
 
     /**
-     * Test key 2.
-     * Unmarshalling always failed.
+     * Test key 2. Unmarshalling always failed.
      */
     public static class TestKeyAlwaysFailed extends TestKey {
-        /** Test key 2 */
+        /** Test key 2. */
         public TestKeyAlwaysFailed(String field) {
             super(field);
         }
 
-        /** Test key 2 */
+        /** Test key 2. */
         public TestKeyAlwaysFailed() {
         }
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            nodeCnt.decrementAndGet();
+            readCnt.decrementAndGet();
             throw new IOException("Class can not be unmarshalled"); //will throw exception on primary node
         }
 
@@ -138,20 +137,23 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
      */
     public void testResponseMessageOnUnmarshallingFailed() {
 
-        nodeCnt.set(1);
+        //Checking failed unmarshalling on primary node.
+        readCnt.set(1);
 
         try {
-            jcache(0).put(new TestKeyAlwaysFailed("1"), "");
+            jcache(0).put(new TestKeyAlwaysFailed("1"), ""); //put will fail at primary node.
+
             assert false : "p2p marshalling failed, but error response was not sent";
         }
         catch (CacheException e) {
             assert X.hasCause(e, IOException.class);
         }
 
-        assert nodeCnt.get() == 0;//put request should not go to backup node in case failed at primary.
+        assert readCnt.get() == 0; //put request should not be handled by backup node in case failed at primary.
 
         try {
             assert jcache(0).get(new TestKeyAlwaysFailed("1")) == null;
+
             assert false : "p2p marshalling failed, but error response was not sent";
         }
         catch (CacheException e) {
@@ -160,20 +162,22 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
         assert grid(0).cachex().entrySet().size() == 0;
 
-        nodeCnt.set(2); //put request will be unmarshalled twice (at primary and at backup node).
+        //Checking failed unmarshalling on backup node.
+        readCnt.set(2); //put request will be unmarshalled twice (at primary and at backup node).
 
         try {
-            jcache(0).put(new TestKey("1"), "");//put will fail at backup node.
+            jcache(0).put(new TestKey("1"), ""); //put will fail at backup node only.
+
             assert false : "p2p marshalling failed, but error response was not sent";
         }
         catch (CacheException e) {
             assert X.hasCause(e, IOException.class);
         }
 
-        assert nodeCnt.get() == 0;//put request should go to primary and backup node.
+        assert readCnt.get() == 0; //put request should be handled by primary and backup node.
 
-        // Need to have to exception while unmarshalling getKeyResponse.
-        nodeCnt.set(3); //get response will me unmarshalled twice (request at primary node and response at client).
+        // Need to have no exception while unmarshalling getKeyResponse.
+        readCnt.set(3); //get response will me unmarshalled twice (request at primary node and response at client).
 
         assert jcache(0).get(new TestKey("1")) == null;
 


[31/36] incubator-ignite git commit: IGNITE-620. Add CI tooling code for patch validation

Posted by sb...@apache.org.
IGNITE-620. Add CI tooling code for patch validation


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

Branch: refs/heads/ignite-695
Commit: 114a8528c6c0030434227a26523e3c16588145c1
Parents: 8c2e03e
Author: Konstantin Boudnik <co...@wandisco.com>
Authored: Thu Mar 26 12:46:19 2015 -0700
Committer: Konstantin Boudnik <co...@wandisco.com>
Committed: Fri May 1 16:49:09 2015 -0700

----------------------------------------------------------------------
 dev-tools/.gitignore                       |   2 +
 dev-tools/build.gradle                     |  46 ++++++++
 dev-tools/src/main/groovy/jiraslurp.groovy | 147 ++++++++++++++++++++++++
 3 files changed, 195 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/114a8528/dev-tools/.gitignore
----------------------------------------------------------------------
diff --git a/dev-tools/.gitignore b/dev-tools/.gitignore
new file mode 100644
index 0000000..3036616
--- /dev/null
+++ b/dev-tools/.gitignore
@@ -0,0 +1,2 @@
+validated-jira.txt
+.gradle

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/114a8528/dev-tools/build.gradle
----------------------------------------------------------------------
diff --git a/dev-tools/build.gradle b/dev-tools/build.gradle
new file mode 100644
index 0000000..30ae6b7
--- /dev/null
+++ b/dev-tools/build.gradle
@@ -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.
+ */
+apply plugin: 'groovy'
+
+repositories {
+    mavenCentral()
+}
+
+dependencies {
+    compile 'org.codehaus.groovy:groovy-all:2.2.1'
+}
+
+task help {
+  println '''There are two interfaces to work with JIRA attachment validation tool
+  - to do the batch validation of all latest patch attachments
+     gradle slurp
+  - to grab a single JIRA's latest attachment and run test validation on it
+     JIRA_NUM=INGITE-### gradle patchapply'''
+}
+
+task slurp (dependsOn: 'classes', type: JavaExec) {
+  args (project.buildDir, 'slurp')
+  main = 'jiraslurp'
+  classpath = sourceSets.main.runtimeClasspath
+}
+
+task patchapply (dependsOn: 'classes', type: JavaExec) {
+  args ("JIRA_NUM=${System.getenv('JIRA_NUM')}")
+  main = 'jiraslurp'
+  classpath = sourceSets.main.runtimeClasspath
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/114a8528/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
new file mode 100644
index 0000000..332686f
--- /dev/null
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -0,0 +1,147 @@
+/*
+ * 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.
+ */
+/**
+ * Parsing a special filter from Apache Ignite JIRA and picking up latest by ID
+ * attachments to process.
+ */
+final GIT_REPO = "https://git1-us-west.apache.org/repos/asf/incubator-ignite.git"
+final ATTACHMENT_URL = "https://issues.apache.org/jira/secure/attachment"
+final validated_filename = "validated-jira.txt"
+final LAST_SUCCESSFUL_ARTIFACT = "guestAuth/repository/download/Ignite_PatchValidation_Main/.lastSuccessful/$validated_filename"
+
+final def JIRA_CMD = System.getProperty('JIRA_COMMAND', 'jira.sh')
+LinkedHashMap<String, String> jirasAttached = [:]
+
+def readHistory = {
+  final int MAX_HISTORY = 5000
+
+  List validated_list = []
+  def validated = new File(validated_filename)
+  if (validated.exists()) {
+    validated_list = validated.text.split('\n')
+    validated.delete()
+  } else {
+    try {
+      validated_list =
+          new URL("http://204.14.53.152/$LAST_SUCCESSFUL_ARTIFACT").text.split('\n')
+    } catch (Exception e) {
+      println e.getMessage()
+    }
+  }
+  // Let's make sure the preserved history isn't too long
+  if (validated_list.size > MAX_HISTORY) {
+    validated_list = validated_list[validated_list.size-MAX_HISTORY..validated_list.size-1]
+  }
+  validated_list
+}
+
+/**
+ * Accepting the <jira> XML element from JIRA stream
+ * @return <code>null</code> or <code>JIRA-###,latest_attach_id</code>
+ */
+def getLatestAttachment = { jira ->
+  def latestAttr = jira.attachments[0].attachment.list().sort {
+    it.@id.toInteger()
+  }.reverse()[0]
+  String row = null
+  if (latestAttr == null) {
+    println "${jira.key} is in invalid state: patch is not available"
+  } else {
+    row = "${jira.key},${latestAttr.@id}"
+  }
+}
+
+def checkForAttachments = {
+  def JIRA_FILTER =
+      "https://issues.apache.org/jira/sr/jira.issueviews:searchrequest-xml/12330308/SearchRequest-12330308.xml?tempMax=100&field=key&field=attachments"
+  def rss = new XmlSlurper().parse(JIRA_FILTER)
+  List list = readHistory{}
+
+  rss.channel.item.each { jira ->
+    String row = getLatestAttachment (jira)
+    if (row != null && !list.contains(row)) {
+      def pair = row.split(',')
+      jirasAttached.put(pair[0] as String, pair[1] as String)
+      list.add(row)
+    }
+  }
+
+  // Write everything back to persist the list
+  def validated = new File(validated_filename)
+  validated << list.join('\n')
+}
+
+def checkprocess = { process ->
+  process.waitFor()
+  if (process.exitValue() != 0) {
+    println "Return code: " + process.exitValue()
+    println "Errout:\n" + process.err.text
+    assert process.exitValue() == 0 || process.exitValue() == 128
+  }
+}
+
+def create_gitbranch = {jira, attachementURL ->
+  println jira
+  GIT_REPO
+  println "$ATTACHMENT_URL/$attachementURL/"
+
+  def patchFile = new File("${jira}.patch")
+  patchFile << new URL("$ATTACHMENT_URL/$attachementURL/").text
+  checkprocess "git clone --depth 1 $GIT_REPO".execute()
+  checkprocess "git checkout -b sprint-2 origin/sprint-2".execute(null, new File('incubator-ignite'))
+  checkprocess "git am ../${patchFile.name}".execute(null, new File('incubator-ignite'))
+  patchFile.delete()
+}
+
+def JIRA_xml = { jiranum ->
+  "https://issues.apache.org/jira/si/jira.issueviews:issue-xml/$jiranum/${jiranum}.xml"
+}
+
+args.each {
+  println it
+  def parameters = it.split('=')
+
+  if (parameters[0] == 'slurp') {
+    checkForAttachments()
+    // For each ticket with new attachment, let's trigger remove build
+    jirasAttached.each { k, v ->
+      //  Trailing slash is important for download; only need to pass JIRA number
+      println "Triggering the build for: $k = $ATTACHMENT_URL/$v/"
+    }
+  } else if (parameters.length == 2 && parameters[0] == 'JIRA_NUM' && parameters[1] ==~ /\w+-\d+/) {
+    // Extract JIRA rss from the and pass the ticket element into attachment extraction
+    def rss = new XmlSlurper().parse(JIRA_xml(parameters[1]))
+    String row = getLatestAttachment(rss.channel.item)
+    if (row != null) {
+      def pair = row.split(',')
+      create_gitbranch(pair[0], pair[1])
+    }
+  }
+}
+
+/* Workflow:
+  1. download an attachment if JIRA num's set; otherwise get all latest attachments not mentioned in the
+     validated-jira.txt file from the last successful build
+  2. trigger test build(s) parametrised by JIRA no.
+  3. test build will download JIRA's latest attachment and apply it to currently checked out repo;
+     - build will fail with comment on JIRA if that can not apply
+     - build will post error/success comment depends on the test results
+*/
+// TODO
+//   - TC's test job needs to send a comment to JIRA
+//       $JIRA_CMD -a addComment -s https://issues.apache.org/jira -u ignite-ci -p ci-of-1gnit3 --issue IGNITE-495 --comment "Trying latest version of the jira-cli"


[22/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: b0374bca8929687f6398076c795b68673fa244ca
Parents: b45679c
Author: avinogradov <av...@gridgain.com>
Authored: Wed Apr 29 15:25:22 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Apr 29 15:25:22 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 13 +++++++
 .../dht/preloader/GridDhtForceKeysResponse.java | 36 ++++++++++++++++----
 2 files changed, 42 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0374bca/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 14d6f7c..a6cbc37 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
@@ -302,6 +302,13 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
+    /**
+     * Sends response on failed message.
+     * @param nodeId node id.
+     * @param res response.
+     * @param cctx shared context.
+     * @param plc grid io policy.
+     */
     private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx,
         GridIoPolicy plc) {
         try {
@@ -313,6 +320,12 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
+    /**
+     * Processes failed messages.
+     * @param nodeId niode id.
+     * @param msg message.
+     * @throws IgniteCheckedException
+     */
     private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0374bca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 5c6e24f..41ce0be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -43,8 +43,12 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
     private IgniteUuid miniId;
 
     /** Error. */
+    @GridDirectTransient
     private volatile IgniteCheckedException err;
 
+    /** Serialized error. */
+    private byte[] errBytes;
+
     /** Missed (not found) keys. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -160,6 +164,8 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
             for (GridCacheEntryInfo info : infos)
                 info.marshal(cctx);
         }
+
+        errBytes = ctx.marshaller().marshal(err);
     }
 
     /** {@inheritDoc} */
@@ -175,6 +181,8 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
             for (GridCacheEntryInfo info : infos)
                 info.unmarshal(cctx, ldr);
         }
+
+        err = ctx.marshaller().unmarshal(errBytes, ldr);
     }
 
     /** {@inheritDoc} */
@@ -193,24 +201,30 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
 
         switch (writer.state()) {
             case 3:
-                if (!writer.writeIgniteUuid("futId", futId))
+                if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeCollection("infos", infos, MessageCollectionItemType.MSG))
+                if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeCollection("infos", infos, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 6:
+                if (!writer.writeIgniteUuid("miniId", miniId))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
                 if (!writer.writeCollection("missedKeys", missedKeys, MessageCollectionItemType.MSG))
                     return false;
 
@@ -233,7 +247,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
 
         switch (reader.state()) {
             case 3:
-                futId = reader.readIgniteUuid("futId");
+                errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -241,7 +255,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
                 reader.incrementState();
 
             case 4:
-                infos = reader.readCollection("infos", MessageCollectionItemType.MSG);
+                futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -249,7 +263,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
                 reader.incrementState();
 
             case 5:
-                miniId = reader.readIgniteUuid("miniId");
+                infos = reader.readCollection("infos", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -257,6 +271,14 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
                 reader.incrementState();
 
             case 6:
+                miniId = reader.readIgniteUuid("miniId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
                 missedKeys = reader.readCollection("missedKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -276,7 +298,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 7;
+        return 8;
     }
 
     /** {@inheritDoc} */


[06/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: 0d982f76671da38e74de80d69522c6f19d67c7ba
Parents: 9a79e70
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 23 15:01:03 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 23 15:01:03 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 37 ++++++--
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  3 +-
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 92 ++++++++++++++++++++
 3 files changed, 125 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d982f76/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 957f3c8..b584b17 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
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.processors.affinity.*;
+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.*;
 import org.apache.ignite.internal.util.*;
@@ -297,9 +298,9 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
-    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheContext ctx) {
+    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx, GridIoPolicy plc) {
         try {
-            ctx.io().send(nodeId, res, ctx.ioPolicy());
+            cctx.io().send(nodeId, res, plc);
         }
         catch (IgniteCheckedException e) {
             U.error(log, "Failed to send response to node (is node still alive?) [nodeId=" + nodeId +
@@ -311,6 +312,18 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
+            case 34:{
+                GridDhtTxPrepareRequest req = (GridDhtTxPrepareRequest)msg;
+
+                GridDhtTxPrepareResponse res = new GridDhtTxPrepareResponse(req.version(), req.futureId(), req.miniId());
+
+                res.error(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, req.policy());
+            }
+
+            break;
+
             case 38: {
                 GridDhtAtomicUpdateRequest req = (GridDhtAtomicUpdateRequest)msg;
 
@@ -318,7 +331,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.onError(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, ctx);
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
             }
 
             break;
@@ -332,7 +345,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.onError(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, ctx);
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
             }
 
             break;
@@ -347,11 +360,25 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(req.classError());
 
-                sendResponseOnFailedMessage(nodeId, res, ctx);
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
             }
 
             break;
 
+            case 55: {
+                GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg;
+
+                GridNearTxPrepareResponse res = new GridNearTxPrepareResponse(req.version(), req.futureId(),
+                    req.miniId(), req.version(), null, null, null);
+
+                res.error(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, req.policy());
+            }
+
+            break;
+
+
             default:
                 throw new IgniteCheckedException("Failed to send response to node. Unsupported direct type [message="
                     + msg + "]");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d982f76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 60f2226..7edbff2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.*;
  */
 public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTest {
     /** Allows to change behavior of readExternal method. */
-    private static AtomicInteger readCnt = new AtomicInteger();
+    protected static AtomicInteger readCnt = new AtomicInteger();
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -136,7 +136,6 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
      * Tests that correct response will be sent to client node in case of unmarshalling failed.
      */
     public void testResponseMessageOnUnmarshallingFailed() {
-
         //Checking failed unmarshalling on primary node.
         readCnt.set(1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d982f76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
new file mode 100644
index 0000000..cb6d444
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
@@ -0,0 +1,92 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.transactions.*;
+
+import java.io.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmarshallingErrorTest {
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /**
+     * Sends put with optimistic lock and handles fail.
+     */
+    protected void failOptimistic() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey("1"), "");
+
+            tx.commit();
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (IgniteException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert readCnt.get() == 0; //ensure we have read counts as expected.
+    }
+
+    /**
+     * Sends put with pessimistic lock and handles fail.
+     */
+    protected void failPessimictic() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey("1"), "");
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (IgniteException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert readCnt.get() == 0; //ensure we have read counts as expected.
+    }
+
+    /**
+     * Tests that correct response will be sent to client node in case of unmarshalling failed.
+     */
+    public void testResponseMessageOnUnmarshallingFailed() {
+//        //GridNearTxPrepareRequest unmarshalling failed test
+//        readCnt.set(2);
+//
+//        failOptimistic();
+//
+//        //GridDhtTxPrepareRequest unmarshalling failed test
+//        readCnt.set(3);
+//
+//        failOptimistic();
+
+//        readCnt.set(1);
+//
+//        failPessimictic();
+
+    }
+}


[02/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: 7a8e075c029fdafb83e4a32871333f3dbe091cf9
Parents: 4843b66 8d13aa1
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 21 14:23:56 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 21 14:23:56 2015 +0300

----------------------------------------------------------------------
 .../examples/ScalarContinuationExample.scala    |   10 +-
 .../cache/eviction/fifo/FifoEvictionPolicy.java |    7 +-
 .../igfs/IgfsPerBlockLruEvictionPolicy.java     |    3 +-
 .../cache/eviction/lru/LruEvictionPolicy.java   |    5 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  431 +++++
 .../sorted/SortedEvictionPolicyMBean.java       |   66 +
 .../cache/eviction/sorted/package-info.java     |   21 +
 .../apache/ignite/cache/query/QueryMetrics.java |    2 +-
 .../ignite/compute/ComputeJobContinuation.java  |    2 +
 .../configuration/CacheConfiguration.java       |    3 +-
 .../ignite/internal/GridJobContextImpl.java     |  100 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   10 +-
 .../apache/ignite/internal/IgniteKernal.java    |   10 +-
 .../internal/managers/GridManagerAdapter.java   |    4 +-
 .../discovery/GridDiscoveryManager.java         |    2 +-
 .../affinity/GridAffinityAssignmentCache.java   |    7 +-
 .../processors/cache/CacheLockImpl.java         |   20 +-
 .../processors/cache/CacheOperationContext.java |  170 ++
 .../processors/cache/CacheProjection.java       | 1386 -------------
 .../internal/processors/cache/GridCache.java    |  223 ---
 .../processors/cache/GridCacheAdapter.java      |  155 +-
 .../cache/GridCacheConcurrentMap.java           |   24 +-
 .../processors/cache/GridCacheContext.java      |   66 +-
 .../processors/cache/GridCacheGateway.java      |   38 +-
 .../processors/cache/GridCacheProcessor.java    |   21 +-
 .../processors/cache/GridCacheProjectionEx.java |  351 ----
 .../cache/GridCacheProjectionImpl.java          |  766 --------
 .../processors/cache/GridCacheProxy.java        |   27 -
 .../processors/cache/GridCacheProxyImpl.java    |  396 ++--
 .../processors/cache/GridCacheUtils.java        |    6 +-
 .../processors/cache/IgniteCacheProxy.java      |  225 ++-
 .../processors/cache/IgniteInternalCache.java   | 1816 ++++++++++++++++++
 .../cache/affinity/GridCacheAffinityProxy.java  |   30 +-
 .../CacheDataStructuresManager.java             |    8 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    7 +-
 .../distributed/near/GridNearAtomicCache.java   |    6 +-
 .../near/GridNearTransactionalCache.java        |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |   10 +-
 .../processors/cache/query/CacheQueries.java    |  103 -
 .../cache/query/CacheQueriesImpl.java           |  142 --
 .../cache/query/CacheQueriesProxy.java          |  200 --
 .../processors/cache/query/CacheQuery.java      |    5 +-
 .../cache/query/GridCacheQueryManager.java      |  110 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |    4 +-
 .../cache/transactions/IgniteTxAdapter.java     |    2 -
 .../transactions/IgniteTxLocalAdapter.java      |   16 +-
 .../cache/transactions/IgniteTxManager.java     |   44 -
 .../datastreamer/DataStreamerCacheUpdaters.java |   13 +-
 .../datastructures/DataStructuresProcessor.java |   24 +-
 .../datastructures/GridCacheAtomicLongImpl.java |    4 +-
 .../GridCacheAtomicReferenceImpl.java           |    4 +-
 .../GridCacheAtomicSequenceImpl.java            |    4 +-
 .../GridCacheAtomicStampedImpl.java             |    4 +-
 .../GridCacheCountDownLatchImpl.java            |    4 +-
 .../datastructures/GridCacheSetImpl.java        |    2 +-
 .../processors/igfs/IgfsDataManager.java        |    6 +-
 .../processors/igfs/IgfsMetaManager.java        |    8 +-
 .../processors/job/GridJobHoldListener.java     |    6 +-
 .../processors/job/GridJobProcessor.java        |   22 +-
 .../internal/processors/job/GridJobWorker.java  |   23 +-
 .../handlers/cache/GridCacheCommandHandler.java |   64 +-
 .../service/GridServiceProcessor.java           |    2 +-
 .../processors/task/GridTaskProcessor.java      |    6 +-
 .../visor/cache/VisorCacheMetadataTask.java     |    9 +-
 .../internal/visor/cache/VisorCacheMetrics.java |    4 +-
 .../visor/cache/VisorCacheRebalanceTask.java    |    4 +-
 .../visor/cache/VisorCacheResetMetricsTask.java |    2 +-
 .../visor/cache/VisorCacheSwapBackupsTask.java  |    2 +-
 .../ignite/startup/BasicWarmupClosure.java      |   20 +-
 .../TransactionSynchronization.java             |   45 -
 .../internal/GridCacheProjectionRemoveTest.java |   41 -
 .../internal/GridContinuousTaskSelfTest.java    |  114 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |    5 +
 .../internal/IgniteInternalCacheRemoveTest.java |   41 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |    2 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |    2 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |    4 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |    4 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |    4 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |   22 +-
 .../IgniteCachePutGetRestartAbstractTest.java   |  234 +++
 .../IgniteCacheTxFairAffinityNodeJoinTest.java  |   35 +
 ...iteCacheTxNearDisabledPutGetRestartTest.java |   30 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    2 +-
 ...micFairAffinityMultiNodeFullApiSelfTest.java |   35 +
 ...achePartitionedPreloadLifecycleSelfTest.java |    2 +-
 ...nedFairAffinityMultiNodeFullApiSelfTest.java |   36 +
 ...dezvousAffinityMultiNodeFullApiSelfTest.java |   35 -
 ...CacheReplicatedPreloadLifecycleSelfTest.java |    2 +-
 ...dCacheSortedBatchEvictionPolicySelfTest.java |  385 ++++
 .../GridCacheSortedEvictionPolicySelfTest.java  |  373 ++++
 .../GridCacheSwapScanQueryAbstractSelfTest.java |   26 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |    2 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |    2 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |   10 +-
 .../GridServiceReassignmentSelfTest.java        |    2 +-
 .../GridContinuousOperationsLoadTest.java       |    3 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    1 -
 .../IgniteCacheEvictionSelfTestSuite.java       |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    3 +
 .../IgniteCacheFullApiSelfTestSuite.java        |    4 +-
 .../ignite/util/TestTcpCommunicationSpi.java    |   54 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |   14 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  |    8 +-
 .../HibernateAccessStrategyAdapter.java         |   10 +-
 .../hibernate/HibernateCollectionRegion.java    |    2 +-
 .../cache/hibernate/HibernateEntityRegion.java  |    2 +-
 .../hibernate/HibernateGeneralDataRegion.java   |    2 +-
 .../hibernate/HibernateNaturalIdRegion.java     |    2 +-
 .../HibernateNonStrictAccessStrategy.java       |    4 +-
 .../hibernate/HibernateQueryResultsRegion.java  |    2 +-
 .../HibernateReadOnlyAccessStrategy.java        |    2 +-
 .../HibernateReadWriteAccessStrategy.java       |    2 +-
 .../ignite/cache/hibernate/HibernateRegion.java |    4 +-
 .../cache/hibernate/HibernateRegionFactory.java |   10 +-
 .../hibernate/HibernateTimestampsRegion.java    |    2 +-
 .../HibernateTransactionalAccessStrategy.java   |    4 +-
 .../HibernateTransactionalDataRegion.java       |    2 +-
 .../HibernateL2CacheConfigurationSelfTest.java  |    2 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |    4 +-
 .../GridCacheAbstractFieldsQuerySelfTest.java   |   53 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   66 +-
 ...idCacheReduceQueryMultithreadedSelfTest.java |    3 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |    3 +
 ...GridCachePartitionedFieldsQuerySelfTest.java |    4 +-
 .../GridCacheReplicatedFieldsQuerySelfTest.java |    8 +-
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |   11 +-
 ...cheReduceFieldsQueryPartitionedSelfTest.java |    4 +-
 .../cache/IgniteCacheAbstractBenchmark.java     |    2 +-
 .../jdbc/IgniteJdbcStoreAbstractBenchmark.java  |    4 +-
 133 files changed, 4822 insertions(+), 4304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a8e075c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7a8e075c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------


[21/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-695
Commit: b45679c197da2a5233f0d12ddd80c60761d9f4ff
Parents: 1f35f3a 77d092c
Author: avinogradov <av...@gridgain.com>
Authored: Wed Apr 29 14:51:02 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Apr 29 14:51:02 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               | 22 +++---
 .../processors/cache/IgniteCacheProxy.java      | 10 +--
 .../distributed/GridCacheLockAbstractTest.java  | 75 ++++++++++++++++++++
 3 files changed, 88 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[36/36] incubator-ignite git commit: Merge branch 'ignite-sprint-5' into ignite-695

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' into ignite-695


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

Branch: refs/heads/ignite-695
Commit: 6ea8f92d4493309728dcc9570a41fb67b5b5923e
Parents: eb98680 6f1d642
Author: Artem Shutak <as...@gridgain.com>
Authored: Wed May 6 15:06:40 2015 +0300
Committer: Artem Shutak <as...@gridgain.com>
Committed: Wed May 6 15:06:40 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  32 +-
 dev-tools/.gitignore                            |   2 +
 dev-tools/build.gradle                          |  45 +++
 dev-tools/src/main/groovy/jiraslurp.groovy      | 146 +++++++++
 examples/pom.xml                                |   2 +-
 modules/aop/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 +-
 .../processors/cache/GridCacheIoManager.java    | 314 +++++++++++++++----
 .../processors/cache/GridCacheMessage.java      |   8 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   8 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   2 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |  18 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   6 +
 .../dht/preloader/GridDhtForceKeysResponse.java |  54 +++-
 .../distributed/near/GridNearGetResponse.java   |   8 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |  20 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   | 189 +++++++++++
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  56 ++++
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  80 +++++
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 109 +++++++
 .../cache/IgniteCachePeekModesAbstractTest.java |  15 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   2 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |   4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |  41 +++
 modules/extdata/p2p/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 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |  11 +-
 .../cache/GridCacheOffHeapSelfTest.java         |  11 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  56 ++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/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/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/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 +-
 pom.xml                                         |   2 +-
 57 files changed, 1160 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6ea8f92d/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------


[27/36] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-646' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-646' into ignite-sprint-5


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

Branch: refs/heads/ignite-695
Commit: 957c206bdea433d9cef1a54d0bba0887d51123e8
Parents: d079850 b0374bc
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 16:13:24 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 16:13:24 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 314 +++++++++++++++----
 .../processors/cache/GridCacheMessage.java      |   8 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   8 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   2 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |  18 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   6 +
 .../dht/preloader/GridDhtForceKeysResponse.java |  54 +++-
 .../distributed/near/GridNearGetResponse.java   |   8 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   | 189 +++++++++++
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  56 ++++
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  80 +++++
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 109 +++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |  41 +++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  56 ++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +
 16 files changed, 875 insertions(+), 78 deletions(-)
----------------------------------------------------------------------



[12/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: 163be30b3b0460967cdba5f8d093eb4ce3c1007a
Parents: db7aa24
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 24 19:42:49 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 24 19:42:49 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheP2pUnmarshallingErrorTest.java          | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/163be30b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index a50f07c..277f10d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -32,7 +32,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /** Allows to change behavior of readExternal method. */
     protected static AtomicInteger readCnt = new AtomicInteger();
 
-    /** iterable key */
+    /** Iterable key. */
     protected static int key = 0;
 
     /** {@inheritDoc} */
@@ -81,7 +81,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         public TestKey() {
         }
 
-        /** field. */
+        /** Field. */
         private String field;
 
         /** {@inheritDoc} */
@@ -160,15 +160,17 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
         failAtomicGet();
 
+        //Check that cache is empty.
         readCnt.set(100);
 
         assert jcache(0).get(new TestKey("1")) == null;
 
+        //GridDhtAtomicUpdateRequest unmarshalling failed test
         readCnt.set(2);
 
-        //GridDhtAtomicUpdateRequest unmarshalling failed test
         failAtomicPut();
 
+        //Check that cache is not empty.
         readCnt.set(100);
 
         assert jcache(0).get(new TestKey("1")) != null;


[11/36] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-695
Commit: db7aa241ac522c9b9720d3c9bd4198fb66b94df2
Parents: 9e4bc10
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 24 19:38:04 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 24 19:38:04 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 15 ++++
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 72 ++++++++------------
 2 files changed, 42 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db7aa241/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 1fe1f50..c9af788 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
@@ -315,6 +315,21 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
+            case 30: {
+                GridDhtLockRequest req = (GridDhtLockRequest)msg;
+
+                GridDhtLockResponse res = new GridDhtLockResponse(
+                    ctx.cacheId(),
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    0);
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
             case 34:{
                 GridDhtTxPrepareRequest req = (GridDhtTxPrepareRequest)msg;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db7aa241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
index 1a42407..06176aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.transactions.*;
 
+import javax.cache.*;
 import java.io.*;
 
 /**
@@ -35,8 +36,14 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
         return CacheAtomicityMode.TRANSACTIONAL;
     }
 
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (!gridName.endsWith("0"))
+            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
+
+        return cfg;
     }
 
     /**
@@ -68,28 +75,9 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
-//        catch (IgniteException e) {
-//            assert X.hasCause(e, IOException.class);
-//        }
-
-        assert readCnt.get() == 0; //ensure we have read count as expected.
-    }
-
-    /**
-     * Sends put with pessimistic lock and handles fail.
-     */
-    protected void failPessimicticOnCommit() {
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-
-            jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
-            tx.commit();
-
-            assert false : "p2p marshalling failed, but error response was not sent";
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
         }
-//        catch (IgniteException e) {
-//            assert X.hasCause(e, IOException.class);
-//        }
 
         assert readCnt.get() == 0; //ensure we have read count as expected.
     }
@@ -98,32 +86,26 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
      * Tests that correct response will be sent to client node in case of unmarshalling failed.
      */
     public void testResponseMessageOnUnmarshallingFailed() {
-//        //GridNearTxPrepareRequest unmarshalling failed test
-//        readCnt.set(2);
-//
-//        failOptimistic();
-//
-//        //GridDhtTxPrepareRequest unmarshalling failed test
-//        readCnt.set(3);
-//
-//        failOptimistic();
-
-//        //GridNearLockRequest unmarshalling failed test
-//        readCnt.set(2);
-//
-//        failPessimictic();
-
-        //? unmarshalling failed test
-        readCnt.set(1000);
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+        //GridNearTxPrepareRequest unmarshalling failed test
+        readCnt.set(2);
 
-            TestKey tstKey = new TestKey(String.valueOf(++key));
-            jcache(0).put(tstKey, "");
-            jcache(0).lock(tstKey).lock();
-        }
+        failOptimistic();
+
+        //GridDhtTxPrepareRequest unmarshalling failed test
+        readCnt.set(3);
 
+        failOptimistic();
 
+        //GridNearLockRequest unmarshalling failed test
+        readCnt.set(2);
 
+        failPessimictic();
 
+        //GridDhtLockRequest unmarshalling failed test
+        readCnt.set(3);
+
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
+        }
     }
 }