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/07/01 16:49:35 UTC

[01/40] incubator-ignite git commit: #IGNITE-1046 Added dockerfile and scripts for running container with ignite.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648-failover 4889b6da7 -> 430c17e2c


#IGNITE-1046 Added dockerfile and scripts for running container with ignite.


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

Branch: refs/heads/ignite-648-failover
Commit: 9de7d248d9ed720179887547772a899beea48488
Parents: 1c66078
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 24 16:26:07 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 24 16:26:07 2015 +0300

----------------------------------------------------------------------
 modules/docker/Dockerfile          | 38 +++++++++++++++++++++++++++++
 modules/docker/README.txt          | 11 +++++++++
 modules/docker/build_users_libs.sh | 23 ++++++++++++++++++
 modules/docker/download_ignite.sh  | 33 ++++++++++++++++++++++++++
 modules/docker/execute.sh          | 42 +++++++++++++++++++++++++++++++++
 modules/docker/run.sh              | 11 +++++++++
 6 files changed, 158 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9de7d248/modules/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/modules/docker/Dockerfile b/modules/docker/Dockerfile
new file mode 100644
index 0000000..2723333
--- /dev/null
+++ b/modules/docker/Dockerfile
@@ -0,0 +1,38 @@
+# Start from a Debian image.
+FROM debian:8
+
+# Install tools.
+RUN apt-get update && apt-get install -y --fix-missing \
+  wget \
+  dstat \
+  maven \
+  git
+
+# Intasll Oracle JDK.
+RUN mkdir /opt/jdk
+
+RUN wget --header "Cookie: oraclelicense=accept-securebackup-cookie" \
+  http://download.oracle.com/otn-pub/java/jdk/7u76-b13/jdk-7u76-linux-x64.tar.gz
+
+RUN tar -zxf jdk-7u76-linux-x64.tar.gz -C /opt/jdk
+
+RUN rm jdk-7u76-linux-x64.tar.gz
+
+RUN update-alternatives --install /usr/bin/java java /opt/jdk/jdk1.7.0_76/bin/java 100
+
+RUN update-alternatives --install /usr/bin/javac javac /opt/jdk/jdk1.7.0_76/bin/javac 100
+
+# Sets java variables.
+ENV JAVA_HOME /opt/jdk/jdk1.7.0_76/
+
+# Create working directory
+RUN mkdir /home/ignite_home
+
+WORKDIR /home/ignite_home
+
+# Copy sh files and set permission
+ADD *.sh ./
+
+RUN chmod +x *.sh
+
+CMD ./run.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9de7d248/modules/docker/README.txt
----------------------------------------------------------------------
diff --git a/modules/docker/README.txt b/modules/docker/README.txt
new file mode 100644
index 0000000..0299d4c
--- /dev/null
+++ b/modules/docker/README.txt
@@ -0,0 +1,11 @@
+Apache Ignite Docker module
+------------------------
+
+Apache Ignite Docker module provides Dockerfile and bash scripts for building docker image.
+
+Build image
+-------------------------------------
+
+cd to ./modules/docker
+
+sudo docker build .
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9de7d248/modules/docker/build_users_libs.sh
----------------------------------------------------------------------
diff --git a/modules/docker/build_users_libs.sh b/modules/docker/build_users_libs.sh
new file mode 100644
index 0000000..24f4420
--- /dev/null
+++ b/modules/docker/build_users_libs.sh
@@ -0,0 +1,23 @@
+#!/bin/bash
+
+if [ -z $GIT_REPO ]; then
+  echo Users git repo is not provided.
+
+  exit 0
+fi
+
+git clone $GIT_REPO user-repo
+
+cd user-repo
+
+if [ ! -z $GIT_BRANCH ]; then
+  git checkout $GIT_BRANCH
+fi
+
+if [ ! -z "$BUILD_CMD" ]; then
+  echo "Starting to execute build command: $BUILD_CMD"
+
+  eval "$BUILD_CMD"
+else
+  mvn clean package
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9de7d248/modules/docker/download_ignite.sh
----------------------------------------------------------------------
diff --git a/modules/docker/download_ignite.sh b/modules/docker/download_ignite.sh
new file mode 100644
index 0000000..6a8617f
--- /dev/null
+++ b/modules/docker/download_ignite.sh
@@ -0,0 +1,33 @@
+#!/bin/bash
+
+function download {
+  wget -O ignite.zip $1
+
+  unzip ignite.zip -d ignite
+
+  rm ignite.zip
+
+  exit 0
+}
+
+if [ ! -z $IGNITE_URL ]; then
+  download $IGNITE_URL
+fi
+
+if [ ! -z $IGNITE_VERSION ]; then
+  if [[ $IGNITE_VERSION  =~ [0-9]*\.[0-9]*\.0 ]]; then
+    download http://apache-mirror.rbc.ru/pub/apache/incubator/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-incubating-bin.zip
+  else
+    download http://www.gridgain.com/media/gridgain-community-fabric-${IGNITE_VERSION}.zip
+  fi
+fi
+
+if [ -z $IGNITE_SOURCE ] || [ $IGNITE_SOURCE = "COMMUNITY" ]; then
+  download http://tiny.cc/updater/download_community.php
+fi
+
+if [ $IGNITE_SOURCE = "APACHE" ]; then
+  download http://tiny.cc/updater/download_ignite.php
+fi
+
+echo "Unsupported IGNITE_SOURCE type: ${IGNITE_SOURCE}"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9de7d248/modules/docker/execute.sh
----------------------------------------------------------------------
diff --git a/modules/docker/execute.sh b/modules/docker/execute.sh
new file mode 100644
index 0000000..2aaaa2d
--- /dev/null
+++ b/modules/docker/execute.sh
@@ -0,0 +1,42 @@
+#!/bin/bash
+
+if [ ! -z "$GIT_REPO" ]; then
+  if [ -z "$LIB_PATTERN" ]; then
+    find user-repo/ -regextype posix-extended -regex "user-repo/target/.*(jar|zip)$" -exec cp {} ignite/*/libs \;
+  else
+    find user-repo/ -regextype posix-extended -regex "$LIB_PATTERN" -exec cp {} ignite/*/libs \;
+  fi
+fi
+
+if [ ! -z "$OPTION_LIBS" ]; then
+  IFS=, LIBS_LIST=("$OPTION_LIBS")
+
+  for lib in ${LIBS_LIST[@]}; do
+    cp -r ./ignite/*/libs/optional/"$lib"/* ./ignite/*/libs
+  done
+fi
+
+# Try to download
+if [ ! -z "$IGNITE_CONFIG" ]; then
+  wget -O ignite-config.xml "$IGNITE_CONFIG" 2>/dev/null
+
+  RETVAL=$?
+
+  [ $RETVAL -eq 0 ] && IGNITE_CONFIG=ignite-config.xml
+
+  [ $RETVAL -ne 0 ] && rm ignite-config.xml && echo "Failed download config: $IGNITE_CONFIG. Try to load config from classpath."
+fi
+
+if [ ! -z "$EXEC_CMD" ]; then
+  echo "Starting to execute command: $EXEC_CMD"
+
+  eval "$EXEC_CMD"
+
+  exit 0
+fi
+
+if [ -z "$IGNITE_CONFIG" ]; then
+  ignite/*/bin/ignite.sh
+else
+  ignite/*/bin/ignite.sh ignite-config.xml
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9de7d248/modules/docker/run.sh
----------------------------------------------------------------------
diff --git a/modules/docker/run.sh b/modules/docker/run.sh
new file mode 100644
index 0000000..42fb29d
--- /dev/null
+++ b/modules/docker/run.sh
@@ -0,0 +1,11 @@
+#!/bin/bash
+
+if [ -z $SKIP_DOWNLOAD ]; then
+  ./download_ignite.sh
+fi
+
+if [ -z $SKIP_BUILD_LIBS ]; then
+  ./build_users_libs.sh
+fi
+
+./execute.sh
\ No newline at end of file


[38/40] incubator-ignite git commit: # ignite-648: loadAll fix and IgniteProcessProxy.java refactoring (constructor, close)

Posted by sb...@apache.org.
# ignite-648: loadAll fix and IgniteProcessProxy.java refactoring (constructor, close)


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

Branch: refs/heads/ignite-648-failover
Commit: ec660e537e55b8f983fbe11b00cc6590125094a5
Parents: ae4e791
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 17:44:15 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 17:44:15 2015 +0300

----------------------------------------------------------------------
 ...heAtomicNearOnlyMultiJvmFullApiSelfTest.java |  2 +-
 .../junits/common/GridCommonAbstractTest.java   | 35 +++++++-
 .../multijvm/IgniteCacheProcessProxy.java       |  4 +-
 .../junits/multijvm/IgniteProcessProxy.java     | 88 ++++++++++++++++----
 4 files changed, 109 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ec660e53/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
index 4a2dcdd..ba01611 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
@@ -22,7 +22,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.*;
 /**
  * Multy Jvm tests.
  */
-public class GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest extends
     GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ec660e53/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 1fc4415..01cb240 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -254,7 +254,40 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param replaceExistingValues Replace existing values.
      * @throws Exception If failed.
      */
-    protected static <K> void loadAll(Cache<K, ?> cache, Set<K> keys, boolean replaceExistingValues) throws Exception {
+    protected static <K> void loadAll(Cache<K, ?> cache, final Set<K> keys, final boolean replaceExistingValues) throws Exception {
+        Ignite ignite = cache.unwrap(Ignite.class);
+
+        if (!(ignite instanceof IgniteProcessProxy))
+            loadAll0(cache, keys, replaceExistingValues);
+        else {
+            IgniteProcessProxy proxy = (IgniteProcessProxy)ignite;
+
+            final UUID id = proxy.getId();
+
+            final String cacheName = cache.getName();
+
+            final Set<Object> keysCp = (Set<Object>)keys;
+
+            proxy.remoteCompute().run(new CAX() {
+                @Override public void applyx() throws IgniteCheckedException {
+                    try {
+                        loadAll0(Ignition.ignite(id).cache(cacheName), keysCp, replaceExistingValues);
+                    }
+                    catch (Exception e) {
+                        throw new IgniteCheckedException(e);
+                    }
+                }
+            });
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param keys Keys.
+     * @param replaceExistingValues Replace existing values.
+     * @throws Exception If failed.
+     */
+    private static <K> void loadAll0(Cache<K, ?> cache, Set<K> keys, boolean replaceExistingValues) throws Exception {
         final AtomicReference<Exception> ex = new AtomicReference<>();
 
         final CountDownLatch latch = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ec660e53/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index a478db0..e23739d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -280,8 +280,8 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override  public void loadAll(Set<? extends K> keys, boolean replaceExistingValues, CompletionListener completionListener) {
-        throw new UnsupportedOperationException("Method should be supported.");
+    @Override  public void loadAll(Set<? extends K> keys, boolean replaceExistingValues, CompletionListener completionLsnr) {
+        throw new UnsupportedOperationException("Oparetion can't be supported automatically.");
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ec660e53/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index b96fe5f..6da478d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.*;
@@ -59,9 +60,6 @@ public class IgniteProcessProxy implements IgniteEx {
     /** Grid id. */
     private final UUID id = UUID.randomUUID();
 
-    /** Remote ignite instance started latch. */
-    private final transient CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
-
     /**
      * @param cfg Configuration.
      * @param log Logger.
@@ -84,17 +82,9 @@ public class IgniteProcessProxy implements IgniteEx {
                 filteredJvmArgs.add(arg);
         }
 
-        locJvmGrid.events().localListen(new IgnitePredicateX<Event>() {
-            @Override public boolean applyx(Event e) {
-                if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
-                    rmtNodeStartedLatch.countDown();
+        final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1);
 
-                    return false;
-                }
-
-                return true;
-            }
-        }, EventType.EVT_NODE_JOINED);
+        locJvmGrid.events().localListen(new NodeStartedListener(id, rmtNodeStartedLatch), EventType.EVT_NODE_JOINED);
 
         proc = GridJavaProcess.exec(
             IgniteNodeRunner.class,
@@ -117,6 +107,36 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /**
+     */
+    private static class NodeStartedListener extends IgnitePredicateX<Event> {
+        /** Id. */
+        private final UUID id;
+
+        /** Remote node started latch. */
+        private final CountDownLatch rmtNodeStartedLatch;
+
+        /**
+         * @param id Id.
+         * @param rmtNodeStartedLatch Remotenode started latch.
+         */
+        NodeStartedListener(UUID id, CountDownLatch rmtNodeStartedLatch) {
+            this.id = id;
+            this.rmtNodeStartedLatch = rmtNodeStartedLatch;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean applyx(Event e) {
+            if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
+                rmtNodeStartedLatch.countDown();
+
+                return false;
+            }
+
+            return true;
+        }
+    }
+
+    /**
      * @param gridName Grid name.
      * @return Instance by name or exception wiil be thrown.
      */
@@ -131,6 +151,15 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /**
+     * For usage in closures.
+     *
+     * @return Ignite instance.
+     */
+    private Ignite igniteById() {
+        return Ignition.ignite(id);
+    }
+
+    /**
      * @param locNodeId ID of local node the requested grid instance is managing.
      * @return An instance of named grid. This method never returns {@code null}.
      * @throws IgniteIllegalStateException Thrown if grid was not properly initialized or grid instance was stopped or
@@ -378,7 +407,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
-        // TODO: CODE: implement.
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -388,7 +417,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
-        throw new UnsupportedOperationException("Transactions are not supported in multi JVM mode.");
+        throw new UnsupportedOperationException("Transactions can't be supported automatically in multi JVM mode.");
     }
 
     /** {@inheritDoc} */
@@ -452,11 +481,38 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public void close() throws IgniteException {
+        final CountDownLatch rmtNodeStoppedLatch = new CountDownLatch(1);
+
+        locJvmGrid.events().localListen(new IgnitePredicateX<Event>() {
+            @Override public boolean applyx(Event e) {
+                if (((DiscoveryEvent)e).eventNode().id().equals(id)) {
+                    rmtNodeStoppedLatch.countDown();
+
+                    return false;
+                }
+
+                return true;
+            }
+        }, EventType.EVT_NODE_LEFT, EventType.EVT_NODE_FAILED);
+
+        compute().run(new IgniteRunnable() {
+            @Override public void run() {
+                igniteById().close();
+            }
+        });
+
+        try {
+            assert U.await(rmtNodeStoppedLatch, 15, TimeUnit.SECONDS) : "NodeId=" + id;
+        }
+        catch (IgniteInterruptedCheckedException e) {
+            throw new IgniteException(e);
+        }
+
         try {
             getProcess().kill();
         }
         catch (Exception e) {
-            e.printStackTrace();
+            X.printerr("Could not kill process after close.", e);
         }
     }
 


[14/40] incubator-ignite git commit: # suppress undeployed message if nothing has

Posted by sb...@apache.org.
# suppress undeployed message if nothing has


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

Branch: refs/heads/ignite-648-failover
Commit: 6929a0cbb382d4b924f127ed65b9fbbe4b232780
Parents: 1d5cfbc
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jun 29 11:08:37 2015 -0700
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jun 29 11:08:37 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheDeploymentManager.java      | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6929a0cb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index ff109ed..677377e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -255,17 +255,17 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
             cacheCtx.near().dht().context().swap().onUndeploy(ldr) :
             cacheCtx.swap().onUndeploy(ldr);
 
-        if (cacheCtx.userCache()) {
+        if (cacheCtx.userCache() && (!keys.isEmpty() || swapUndeployCnt != 0)) {
             U.quietAndWarn(log, "");
             U.quietAndWarn(
                 log,
-                "Cleared all cache entries for undeployed class loader [[cacheName=" + cacheCtx.namexx() +
+                "Cleared all cache entries for undeployed class loader [cacheName=" + cacheCtx.namexx() +
                     ", undeployCnt=" + keys.size() + ", swapUndeployCnt=" + swapUndeployCnt +
-                    ", clsLdr=" + ldr.getClass().getName() + ']',
-                "Cleared all cache entries for undeployed class loader for cache: " + cacheCtx.namexx());
+                    ", clsLdr=" + ldr.getClass().getName() + ']');
             U.quietAndWarn(
                 log,
-                "  ^-- Cache auto-undeployment happens in SHARED deployment mode (to turn off, switch to CONTINUOUS mode)");
+                "  ^-- Cache auto-undeployment happens in SHARED deployment mode " +
+                    "(to turn off, switch to CONTINUOUS mode)");
             U.quietAndWarn(log, "");
         }
 


[22/40] incubator-ignite git commit: IGNITE-1063 Added generic for chaining setters.

Posted by sb...@apache.org.
IGNITE-1063 Added generic for chaining setters.


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

Branch: refs/heads/ignite-648-failover
Commit: ef7a35ce9afb830c2e02b624eb3f3298f66e6129
Parents: bade9f1
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 30 12:31:13 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 30 12:31:13 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 105 ++++++++++---------
 .../configuration/NearCacheConfiguration.java   |  10 +-
 2 files changed, 61 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef7a35ce/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 7af4974..e2308f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -419,7 +419,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param name Cache name. May be <tt>null</tt>, but may not be empty string.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setName(String name) {
+    public CacheConfiguration<K, V> setName(String name) {
         A.ensure(name == null || !name.isEmpty(), "Name cannot be null or empty.");
 
         this.name = name;
@@ -444,7 +444,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictPlc Cache expiration policy.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictionPolicy(@Nullable EvictionPolicy evictPlc) {
+    public CacheConfiguration<K, V> setEvictionPolicy(@Nullable EvictionPolicy evictPlc) {
         this.evictPlc = evictPlc;
 
         return this;
@@ -461,7 +461,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param nearCfg Near cache configuration.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setNearConfiguration(NearCacheConfiguration<K, V> nearCfg) {
+    public CacheConfiguration<K, V> setNearConfiguration(NearCacheConfiguration<K, V> nearCfg) {
         this.nearCfg = nearCfg;
 
         return this;
@@ -483,7 +483,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param writeSync Write synchronization mode.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setWriteSynchronizationMode(CacheWriteSynchronizationMode writeSync) {
+    public CacheConfiguration<K, V> setWriteSynchronizationMode(CacheWriteSynchronizationMode writeSync) {
         this.writeSync = writeSync;
 
         return this;
@@ -504,7 +504,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param nodeFilter Predicate specifying on which nodes the cache should be started.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setNodeFilter(IgnitePredicate<ClusterNode> nodeFilter) {
+    public CacheConfiguration<K, V> setNodeFilter(IgnitePredicate<ClusterNode> nodeFilter) {
         this.nodeFilter = nodeFilter;
 
         return this;
@@ -536,7 +536,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictSync {@code true} if synchronized, {@code false} if not.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictSynchronized(boolean evictSync) {
+    public CacheConfiguration<K, V> setEvictSynchronized(boolean evictSync) {
         this.evictSync = evictSync;
 
         return this;
@@ -559,7 +559,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictKeyBufSize Eviction key buffer size.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictSynchronizedKeyBufferSize(int evictKeyBufSize) {
+    public CacheConfiguration<K, V> setEvictSynchronizedKeyBufferSize(int evictKeyBufSize) {
         this.evictKeyBufSize = evictKeyBufSize;
 
         return this;
@@ -593,7 +593,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictSyncConcurrencyLvl Concurrency level for synchronized evictions.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictSynchronizedConcurrencyLevel(int evictSyncConcurrencyLvl) {
+    public CacheConfiguration<K, V> setEvictSynchronizedConcurrencyLevel(int evictSyncConcurrencyLvl) {
         this.evictSyncConcurrencyLvl = evictSyncConcurrencyLvl;
 
         return this;
@@ -619,7 +619,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictSyncTimeout Timeout for synchronized evictions.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictSynchronizedTimeout(long evictSyncTimeout) {
+    public CacheConfiguration<K, V> setEvictSynchronizedTimeout(long evictSyncTimeout) {
         this.evictSyncTimeout = evictSyncTimeout;
 
         return this;
@@ -651,7 +651,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictMaxOverflowRatio Maximum eviction overflow ratio.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictMaxOverflowRatio(float evictMaxOverflowRatio) {
+    public CacheConfiguration<K, V> setEvictMaxOverflowRatio(float evictMaxOverflowRatio) {
         this.evictMaxOverflowRatio = evictMaxOverflowRatio;
 
         return this;
@@ -680,7 +680,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param evictFilter Eviction filter.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEvictionFilter(EvictionFilter<K, V> evictFilter) {
+    public CacheConfiguration<K, V> setEvictionFilter(EvictionFilter<K, V> evictFilter) {
         this.evictFilter = evictFilter;
 
         return this;
@@ -709,7 +709,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #isEagerTtl()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setEagerTtl(boolean eagerTtl) {
+    public CacheConfiguration<K, V> setEagerTtl(boolean eagerTtl) {
         this.eagerTtl = eagerTtl;
 
         return this;
@@ -731,7 +731,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param startSize Cache start size.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setStartSize(int startSize) {
+    public CacheConfiguration<K, V> setStartSize(int startSize) {
         this.startSize = startSize;
 
         return this;
@@ -775,7 +775,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param loadPrevVal Load previous value flag.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setLoadPreviousValue(boolean loadPrevVal) {
+    public CacheConfiguration<K, V> setLoadPreviousValue(boolean loadPrevVal) {
         this.loadPrevVal = loadPrevVal;
 
         return this;
@@ -798,7 +798,8 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @return {@code this} for chaining.
      */
     @SuppressWarnings("unchecked")
-    public CacheConfiguration setCacheStoreFactory(Factory<? extends CacheStore<? super K, ? super V>> storeFactory) {
+    public CacheConfiguration<K, V> setCacheStoreFactory(
+        Factory<? extends CacheStore<? super K, ? super V>> storeFactory) {
         this.storeFactory = storeFactory;
 
         return this;
@@ -819,7 +820,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param aff Cache key affinity.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setAffinity(AffinityFunction aff) {
+    public CacheConfiguration<K, V> setAffinity(AffinityFunction aff) {
         this.aff = aff;
 
         return this;
@@ -842,7 +843,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param cacheMode Caching mode.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setCacheMode(CacheMode cacheMode) {
+    public CacheConfiguration<K, V> setCacheMode(CacheMode cacheMode) {
         this.cacheMode = cacheMode;
 
         return this;
@@ -865,7 +866,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param atomicityMode Cache atomicity mode.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setAtomicityMode(CacheAtomicityMode atomicityMode) {
+    public CacheConfiguration<K, V> setAtomicityMode(CacheAtomicityMode atomicityMode) {
         this.atomicityMode = atomicityMode;
 
         return this;
@@ -888,7 +889,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param atomicWriteOrderMode Cache write ordering mode.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setAtomicWriteOrderMode(CacheAtomicWriteOrderMode atomicWriteOrderMode) {
+    public CacheConfiguration<K, V> setAtomicWriteOrderMode(CacheAtomicWriteOrderMode atomicWriteOrderMode) {
         this.atomicWriteOrderMode = atomicWriteOrderMode;
 
         return this;
@@ -913,7 +914,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param backups Number of backup nodes for one partition.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setBackups(int backups) {
+    public CacheConfiguration<K, V> setBackups(int backups) {
         this.backups = backups;
 
         return this;
@@ -935,7 +936,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param dfltLockTimeout Default lock timeout.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setDefaultLockTimeout(long dfltLockTimeout) {
+    public CacheConfiguration<K, V> setDefaultLockTimeout(long dfltLockTimeout) {
         this.dfltLockTimeout = dfltLockTimeout;
 
         return this;
@@ -956,7 +957,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param invalidate Flag to set this cache into invalidation-based mode. Default value is {@code false}.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setInvalidate(boolean invalidate) {
+    public CacheConfiguration<K, V> setInvalidate(boolean invalidate) {
         this.invalidate = invalidate;
 
         return this;
@@ -978,7 +979,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *      receive JTA transaction manager.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setTransactionManagerLookupClassName(String tmLookupClsName) {
+    public CacheConfiguration<K, V> setTransactionManagerLookupClassName(String tmLookupClsName) {
         this.tmLookupClsName = tmLookupClsName;
 
         return this;
@@ -990,7 +991,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param rebalanceMode Rebalance mode.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceMode(CacheRebalanceMode rebalanceMode) {
+    public CacheConfiguration<K, V> setRebalanceMode(CacheRebalanceMode rebalanceMode) {
         this.rebalanceMode = rebalanceMode;
 
         return this;
@@ -1033,7 +1034,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getRebalanceOrder()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceOrder(int rebalanceOrder) {
+    public CacheConfiguration<K, V> setRebalanceOrder(int rebalanceOrder) {
         this.rebalanceOrder = rebalanceOrder;
 
         return this;
@@ -1057,7 +1058,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param rebalanceBatchSize Rebalance batch size.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceBatchSize(int rebalanceBatchSize) {
+    public CacheConfiguration<K, V> setRebalanceBatchSize(int rebalanceBatchSize) {
         this.rebalanceBatchSize = rebalanceBatchSize;
 
         return this;
@@ -1079,7 +1080,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param swapEnabled {@code True} if swap storage is enabled.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setSwapEnabled(boolean swapEnabled) {
+    public CacheConfiguration<K, V> setSwapEnabled(boolean swapEnabled) {
         this.swapEnabled = swapEnabled;
 
         return this;
@@ -1108,7 +1109,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getMaxConcurrentAsyncOperations()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setMaxConcurrentAsyncOperations(int maxConcurrentAsyncOps) {
+    public CacheConfiguration<K, V> setMaxConcurrentAsyncOperations(int maxConcurrentAsyncOps) {
         this.maxConcurrentAsyncOps = maxConcurrentAsyncOps;
 
         return this;
@@ -1131,7 +1132,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param writeBehindEnabled {@code true} if write-behind is enabled.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setWriteBehindEnabled(boolean writeBehindEnabled) {
+    public CacheConfiguration<K, V> setWriteBehindEnabled(boolean writeBehindEnabled) {
         this.writeBehindEnabled = writeBehindEnabled;
 
         return this;
@@ -1159,7 +1160,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getWriteBehindFlushSize()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setWriteBehindFlushSize(int writeBehindFlushSize) {
+    public CacheConfiguration<K, V> setWriteBehindFlushSize(int writeBehindFlushSize) {
         this.writeBehindFlushSize = writeBehindFlushSize;
 
         return this;
@@ -1188,7 +1189,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getWriteBehindFlushFrequency()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setWriteBehindFlushFrequency(long writeBehindFlushFreq) {
+    public CacheConfiguration<K, V> setWriteBehindFlushFrequency(long writeBehindFlushFreq) {
         this.writeBehindFlushFreq = writeBehindFlushFreq;
 
         return this;
@@ -1215,7 +1216,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getWriteBehindFlushThreadCount()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setWriteBehindFlushThreadCount(int writeBehindFlushThreadCnt) {
+    public CacheConfiguration<K, V> setWriteBehindFlushThreadCount(int writeBehindFlushThreadCnt) {
         this.writeBehindFlushThreadCnt = writeBehindFlushThreadCnt;
 
         return this;
@@ -1242,7 +1243,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getWriteBehindBatchSize()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setWriteBehindBatchSize(int writeBehindBatchSize) {
+    public CacheConfiguration<K, V> setWriteBehindBatchSize(int writeBehindBatchSize) {
         this.writeBehindBatchSize = writeBehindBatchSize;
 
         return this;
@@ -1267,7 +1268,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param rebalancePoolSize Size of rebalancing thread pool.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceThreadPoolSize(int rebalancePoolSize) {
+    public CacheConfiguration<K, V> setRebalanceThreadPoolSize(int rebalancePoolSize) {
         this.rebalancePoolSize = rebalancePoolSize;
 
         return this;
@@ -1290,7 +1291,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param rebalanceTimeout Rebalance timeout (ms).
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceTimeout(long rebalanceTimeout) {
+    public CacheConfiguration<K, V> setRebalanceTimeout(long rebalanceTimeout) {
         this.rebalanceTimeout = rebalanceTimeout;
 
         return this;
@@ -1328,7 +1329,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param rebalanceDelay Rebalance delay to set.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceDelay(long rebalanceDelay) {
+    public CacheConfiguration<K, V> setRebalanceDelay(long rebalanceDelay) {
         this.rebalanceDelay = rebalanceDelay;
 
         return this;
@@ -1364,7 +1365,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * {@code 0} to disable throttling.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setRebalanceThrottle(long rebalanceThrottle) {
+    public CacheConfiguration<K, V> setRebalanceThrottle(long rebalanceThrottle) {
         this.rebalanceThrottle = rebalanceThrottle;
 
         return this;
@@ -1391,7 +1392,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param affMapper Affinity mapper.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setAffinityMapper(AffinityKeyMapper affMapper) {
+    public CacheConfiguration<K, V> setAffinityMapper(AffinityKeyMapper affMapper) {
         this.affMapper = affMapper;
 
         return this;
@@ -1438,7 +1439,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param offHeapMaxMem Maximum memory in bytes available to off-heap memory space.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setOffHeapMaxMemory(long offHeapMaxMem) {
+    public CacheConfiguration<K, V> setOffHeapMaxMemory(long offHeapMaxMem) {
         this.offHeapMaxMem = offHeapMaxMem;
 
         return this;
@@ -1462,7 +1463,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param memMode Memory mode.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setMemoryMode(CacheMemoryMode memMode) {
+    public CacheConfiguration<K, V> setMemoryMode(CacheMemoryMode memMode) {
         this.memMode = memMode;
 
         return this;
@@ -1484,7 +1485,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param interceptor Cache interceptor.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setInterceptor(CacheInterceptor<K, V> interceptor) {
+    public CacheConfiguration<K, V> setInterceptor(CacheInterceptor<K, V> interceptor) {
         this.interceptor = interceptor;
 
         return this;
@@ -1505,7 +1506,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param typeMeta Collection of type metadata.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setTypeMetadata(Collection<CacheTypeMetadata> typeMeta) {
+    public CacheConfiguration<K, V> setTypeMetadata(Collection<CacheTypeMetadata> typeMeta) {
         this.typeMeta = typeMeta;
 
         return this;
@@ -1530,7 +1531,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param readFromBackup {@code true} to allow reads from backups.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setReadFromBackup(boolean readFromBackup) {
+    public CacheConfiguration<K, V> setReadFromBackup(boolean readFromBackup) {
         this.readFromBackup = readFromBackup;
 
         return this;
@@ -1554,7 +1555,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #isCopyOnRead
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setCopyOnRead(boolean cpOnRead) {
+    public CacheConfiguration<K, V> setCopyOnRead(boolean cpOnRead) {
         this.cpOnRead = cpOnRead;
 
         return this;
@@ -1567,7 +1568,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param cls One or more classes with SQL functions.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setSqlFunctionClasses(Class<?>... cls) {
+    public CacheConfiguration<K, V> setSqlFunctionClasses(Class<?>... cls) {
         this.sqlFuncCls = cls;
 
         return this;
@@ -1598,7 +1599,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param longQryWarnTimeout Timeout in milliseconds.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setLongQueryWarningTimeout(long longQryWarnTimeout) {
+    public CacheConfiguration<K, V> setLongQueryWarningTimeout(long longQryWarnTimeout) {
         this.longQryWarnTimeout = longQryWarnTimeout;
 
         return this;
@@ -1623,7 +1624,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param sqlEscapeAll Flag value.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setSqlEscapeAll(boolean sqlEscapeAll) {
+    public CacheConfiguration<K, V> setSqlEscapeAll(boolean sqlEscapeAll) {
         this.sqlEscapeAll = sqlEscapeAll;
 
         return this;
@@ -1660,7 +1661,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param indexedTypes Key and value type pairs.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setIndexedTypes(Class<?>... indexedTypes) {
+    public CacheConfiguration<K, V> setIndexedTypes(Class<?>... indexedTypes) {
         A.ensure(indexedTypes == null || (indexedTypes.length & 1) == 0,
             "Number of indexed types is expected to be even. Refer to method javadoc for details.");
 
@@ -1699,7 +1700,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #setOffHeapMaxMemory(long)
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setSqlOnheapRowCacheSize(int size) {
+    public CacheConfiguration<K, V> setSqlOnheapRowCacheSize(int size) {
         this.sqlOnheapRowCacheSize = size;
 
         return this;
@@ -1720,7 +1721,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param pluginCfgs Cache plugin configurations.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setPluginConfigurations(CachePluginConfiguration... pluginCfgs) {
+    public CacheConfiguration<K, V> setPluginConfigurations(CachePluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;
 
         return this;
@@ -1739,7 +1740,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param topValidator validator.
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration setTopologyValidator(TopologyValidator topValidator) {
+    public CacheConfiguration<K, V> setTopologyValidator(TopologyValidator topValidator) {
         this.topValidator = topValidator;
 
         return this;
@@ -1766,7 +1767,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @return {@code this} for chaining.
      * @see CacheStoreSessionListener
      */
-    public CacheConfiguration setCacheStoreSessionListenerFactories(
+    public CacheConfiguration<K, V> setCacheStoreSessionListenerFactories(
         Factory<? extends CacheStoreSessionListener>... storeSesLsnrs) {
         this.storeSesLsnrs = storeSesLsnrs;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef7a35ce/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
index 0593601..580010a 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
@@ -64,9 +64,12 @@ public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /**
      * @param nearEvictPlc Near eviction policy.
+     * @return {@code this} for chaining.
      */
-    public void setNearEvictionPolicy(EvictionPolicy<K, V> nearEvictPlc) {
+    public NearCacheConfiguration<K, V> setNearEvictionPolicy(EvictionPolicy<K, V> nearEvictPlc) {
         this.nearEvictPlc = nearEvictPlc;
+
+        return this;
     }
 
     /**
@@ -83,9 +86,12 @@ public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Start size for near cache. This property is only used for {@link CacheMode#PARTITIONED} caching mode.
      *
      * @param nearStartSize Start size for near cache.
+     * @return {@code this} for chaining.
      */
-    public void setNearStartSize(int nearStartSize) {
+    public NearCacheConfiguration<K, V> setNearStartSize(int nearStartSize) {
         this.nearStartSize = nearStartSize;
+
+        return this;
     }
 
     /** {@inheritDoc} */


[07/40] incubator-ignite git commit: # ignite-gg-10453 removed test (test is invalid after fix)

Posted by sb...@apache.org.
# ignite-gg-10453 removed test (test is invalid after fix)


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

Branch: refs/heads/ignite-648-failover
Commit: d51db291037866b03ac828430cfdcc914c2f79ed
Parents: d86882c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 29 09:21:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 29 09:21:05 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 -------------------
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 2 files changed, 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d51db291/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
deleted file mode 100644
index c56ad1c..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
+++ /dev/null
@@ -1,119 +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.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.util.concurrent.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Test cache operations with daemon node.
- */
-public class GridCacheDaemonNodeStopSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Daemon flag. */
-    protected boolean daemon;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        daemon = false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        c.setDaemon(daemon);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        c.setConnectorConfiguration(null);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(LOCAL);
-        cc.setAtomicityMode(ATOMIC);
-
-        TransactionConfiguration cfg = new TransactionConfiguration();
-
-        c.setTransactionConfiguration(cfg);
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStartStop() throws Exception {
-        try {
-            daemon = true;
-
-            final IgniteEx daemonNode = startGrid(0);
-
-            final IgniteInternalFuture<Object> f = GridTestUtils.runAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    daemonNode.context().marshallerContext().getClass(1,
-                        GridCacheDaemonNodeStopSelfTest.class.getClassLoader());
-
-                    return null;
-                }
-            });
-
-            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    f.get(300);
-
-                    return null;
-                }
-            }, IgniteFutureTimeoutCheckedException.class);
-
-            // Stop grid.
-            stopGrid(0);
-
-            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    f.get(5, TimeUnit.SECONDS);
-
-                    return null;
-                }
-            }, IllegalStateException.class);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d51db291/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index b83f50f..5947d33 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -100,7 +100,6 @@ public class IgniteCacheTestSuite3 extends TestSuite {
         suite.addTestSuite(GridCacheDaemonNodeLocalSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodePartitionedSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodeReplicatedSelfTest.class);
-        suite.addTestSuite(GridCacheDaemonNodeStopSelfTest.class);
 
         // Write-behind.
         suite.addTest(IgniteCacheWriteBehindTestSuite.suite());


[27/40] incubator-ignite git commit: Merge branch 'ignite-sprint-7' into ignite-1063

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-7' into ignite-1063


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

Branch: refs/heads/ignite-648-failover
Commit: 28525acbe2e19f5605137dbb7e9b3824eab25987
Parents: ef7a35c 6ebcb6d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 30 16:56:15 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 30 16:56:15 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  26 +-
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 3 files changed, 310 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[28/40] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1064' into ignite-sprint-7

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


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

Branch: refs/heads/ignite-648-failover
Commit: e91bc484e224ce8a039ef5bba71aa7f093dce262
Parents: 28525ac 0ef74a1
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 30 17:31:22 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 30 17:31:22 2015 +0300

----------------------------------------------------------------------
 .../processors/clock/GridClockServer.java       | 21 ++++--
 .../TcpDiscoveryMulticastIpFinder.java          | 74 ++++++++++++++++----
 2 files changed, 76 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[10/40] incubator-ignite git commit: minor fix

Posted by sb...@apache.org.
minor fix


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

Branch: refs/heads/ignite-648-failover
Commit: a226666663f0d5de94284a3662a6103c1d9c2b17
Parents: f45f96d
Author: avinogradov <av...@gridgain.com>
Authored: Mon Jun 29 15:55:41 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Jun 29 15:55:41 2015 +0300

----------------------------------------------------------------------
 pom.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a2266666/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b935b2e..4f2441c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -604,12 +604,12 @@
                                         </copy>
 
                                         <!-- appending filename to md5 and sha1 files. to be improved. -->
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true"> ${project.artifactId}-fabric-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true"> ${project.artifactId}-fabric-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true"> ${project.artifactId}-hadoop-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true"> ${project.artifactId}-hadoop-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true"> ${project.artifactId}-${project.version}-src.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true"> ${project.artifactId}-${project.version}-src.zip</concat>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip"/>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip"/>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip"/>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip"/>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message=" ${project.artifactId}-${project.version}-src.zip"/>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message=" ${project.artifactId}-${project.version}-src.zip"/>
 
                                         <copy file="${basedir}/KEYS" todir="${basedir}/target/site" failonerror="false" />
                                     </target>


[15/40] incubator-ignite git commit: # fixed delays on node stop

Posted by sb...@apache.org.
# fixed delays on node stop


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

Branch: refs/heads/ignite-648-failover
Commit: 68c21ac4c924caac6b6e4f7dd287ba1326f4fc11
Parents: 6929a0c
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jun 29 11:44:17 2015 -0700
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jun 29 11:44:17 2015 -0700

----------------------------------------------------------------------
 .../util/ipc/shmem/IpcSharedMemoryServerEndpoint.java     | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68c21ac4/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
index 102c5b1..8c3f29e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
@@ -531,7 +531,9 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
 
             while (true) {
                 try {
-                    Thread.sleep(GC_FREQ);
+                    // Sleep only if not cancelled.
+                    if (lastRunNeeded)
+                        Thread.sleep(GC_FREQ);
                 }
                 catch (InterruptedException ignored) {
                     // No-op.
@@ -559,8 +561,12 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
                 }
 
                 if (isCancelled()) {
-                    if (lastRunNeeded)
+                    if (lastRunNeeded) {
                         lastRunNeeded = false;
+
+                        // Clear interrupted status.
+                        Thread.interrupted();
+                    }
                     else {
                         Thread.currentThread().interrupt();
 


[33/40] incubator-ignite git commit: 1.2.1-SNAPSHOT

Posted by sb...@apache.org.
1.2.1-SNAPSHOT


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

Branch: refs/heads/ignite-648-failover
Commit: 204823c3300ca5c79c2972a8cb3e8dab044ac8d3
Parents: c7e7487
Author: Ignite Teamcity <ig...@apache.org>
Authored: Wed Jul 1 13:04:20 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Wed Jul 1 13:04:20 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/core/src/main/resources/ignite.properties |  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/mesos/pom.xml                             |  2 +-
 modules/rest-http/pom.xml                         |  2 +-
 modules/scalar-2.10/pom.xml                       |  2 +-
 modules/scalar/pom.xml                            |  2 +-
 modules/schedule/pom.xml                          |  2 +-
 modules/schema-import/pom.xml                     |  2 +-
 modules/slf4j/pom.xml                             |  2 +-
 modules/spark-2.10/pom.xml                        |  2 +-
 modules/spark/pom.xml                             |  2 +-
 modules/spring/pom.xml                            |  2 +-
 modules/ssh/pom.xml                               |  2 +-
 modules/tools/pom.xml                             |  2 +-
 modules/urideploy/pom.xml                         |  2 +-
 modules/visor-console-2.10/pom.xml                |  2 +-
 modules/visor-console/pom.xml                     |  2 +-
 modules/visor-plugins/pom.xml                     |  2 +-
 modules/web/pom.xml                               |  2 +-
 modules/yardstick/pom.xml                         |  2 +-
 pom.xml                                           | 14 +++++++-------
 37 files changed, 43 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index bcff11e..2f292e9 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 5eea18d..704358f 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 05ecbc1..38d300e 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 7c4d2c8..ca68ef8 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index bb32689..d752588 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 1ea5fe5..9a09a69 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 79fbc2f..c7cb953 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index b25b2ca..830ca22 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.1.6-SNAPSHOT
+ignite.version=1.2.1-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 26f2e50..0ee1d9b 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.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 1690ab9..f2b3936 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.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 127c7a4..f132d39 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index e28c139..10035c6 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index e300ba0..bb46b2a 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index d001b02..3130f4b 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index b044092..8c02e02 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index b764c96..9f65eb0 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index 531e6f2..2d4df0b 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index af62389..3343438 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 07bee9e..491667d 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 2d7fe6d..c4cb62d 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.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index c4b8d60..b7c9178 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index a048939..7a2e243 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 735035f..49b75bd 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 2898327..7f8fff4 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.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index fb7c86e..01ad28f 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 1b8b04e..7463d52 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 8c00ec8..265bb16 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 35638fe..ab98384 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 23ae1df..708be4f 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 9e5685d..17ba197 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 0add261..76a862b 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index 8806438..193e676 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index 8c6e4c9..87fa4f9 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.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index e4162c0..8a80009 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.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index e6a23bd..f109880 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 5307447..a9ee178 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/204823c3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4f2441c..a6d1609 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.1.6-SNAPSHOT</version>
+    <version>1.2.1-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>
@@ -604,12 +604,12 @@
                                         </copy>
 
                                         <!-- appending filename to md5 and sha1 files. to be improved. -->
-                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip"/>
-                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip"/>
-                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip"/>
-                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip"/>
-                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message=" ${project.artifactId}-${project.version}-src.zip"/>
-                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message=" ${project.artifactId}-${project.version}-src.zip"/>
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message=" ${project.artifactId}-${project.version}-src.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message=" ${project.artifactId}-${project.version}-src.zip" />
 
                                         <copy file="${basedir}/KEYS" todir="${basedir}/target/site" failonerror="false" />
                                     </target>


[21/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-7' into ignite-sprint-7

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


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

Branch: refs/heads/ignite-648-failover
Commit: bade9f16b64416b4ea032699a8a651d0a3ad064f
Parents: 596c538 8c55f8a
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 30 10:39:09 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 30 10:39:09 2015 +0700

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../discovery/GridDiscoveryManager.java         |   6 +
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++++--
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../core/src/test/config/spark/spark-config.xml |  46 +++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ------------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../org/apache/ignite/spark/IgniteContext.scala |  50 ++++-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 ++
 pom.xml                                         |  12 +-
 13 files changed, 380 insertions(+), 149 deletions(-)
----------------------------------------------------------------------



[25/40] incubator-ignite git commit: IGNITE-1062 Split in topology snapshot server and client nodes.

Posted by sb...@apache.org.
IGNITE-1062 Split in topology snapshot server and client nodes.


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

Branch: refs/heads/ignite-648-failover
Commit: 8006a84344118f39e9190643420512fbee06bc73
Parents: bade9f1
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 30 16:06:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 30 16:06:28 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  46 +++-
 .../IgniteTopologyPrintFormatSelfTest.java      | 233 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 3 files changed, 274 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8006a843/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 5e7600f..1d555e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -98,6 +98,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
     };
 
+    /** Predicate filtering client nodes. */
+    private static final IgnitePredicate<ClusterNode> clientFilter = new P1<ClusterNode>() {
+        @Override public boolean apply(ClusterNode n) {
+            return n.isClient();
+        }
+    };
+
     /** Disco history entries comparator. */
     private static final Comparator<Map.Entry<AffinityTopologyVersion, DiscoCache>> histCmp =
         new Comparator<Map.Entry<AffinityTopologyVersion, DiscoCache>>() {
@@ -933,6 +940,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         Collection<ClusterNode> rmtNodes = discoCache.remoteNodes();
 
+        Collection<ClusterNode> serverNodes = discoCache.serverNodes();
+
+        Collection<ClusterNode> clientNodes = discoCache.clientNodes();
+
         ClusterNode locNode = discoCache.localNode();
 
         Collection<ClusterNode> allNodes = discoCache.allNodes();
@@ -949,7 +960,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         double heap = U.heapSize(allNodes, 2);
 
         if (log.isQuiet())
-            U.quiet(false, topologySnapshotMessage(rmtNodes.size(), totalCpus, heap));
+            U.quiet(false, topologySnapshotMessage(serverNodes.size(), clientNodes.size(), totalCpus, heap));
 
         if (log.isDebugEnabled()) {
             String dbg = "";
@@ -959,7 +970,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 ">>> " + PREFIX + "." + U.nl() +
                 ">>> +----------------+" + U.nl() +
                 ">>> Grid name: " + (ctx.gridName() == null ? "default" : ctx.gridName()) + U.nl() +
-                ">>> Number of nodes: " + (rmtNodes.size() + 1) + U.nl() +
+                ">>> Number of server nodes: " + serverNodes.size() + U.nl() +
+                ">>> Number of client nodes: " + clientNodes.size() + U.nl() +
                 (discoOrdered ? ">>> Topology version: " + topVer + U.nl() : "") +
                 ">>> Topology hash: 0x" + Long.toHexString(hash).toUpperCase() + U.nl();
 
@@ -992,19 +1004,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             log.debug(dbg);
         }
         else if (log.isInfoEnabled())
-            log.info(topologySnapshotMessage(rmtNodes.size(), totalCpus, heap));
+            log.info(topologySnapshotMessage(serverNodes.size(), clientNodes.size(), totalCpus, heap));
     }
 
     /**
-     * @param rmtNodesNum Remote nodes number.
+     * @param serverNodesNum Server nodes number.
+     * @param clientNodesNum Client nodes number.
      * @param totalCpus Total cpu number.
      * @param heap Heap size.
      * @return Topology snapshot message.
      */
-    private String topologySnapshotMessage(int rmtNodesNum, int totalCpus, double heap) {
+    private String topologySnapshotMessage(int serverNodesNum, int clientNodesNum, int totalCpus, double heap) {
         return PREFIX + " [" +
             (discoOrdered ? "ver=" + topSnap.get().topVer.topologyVersion() + ", " : "") +
-            "nodes=" + (rmtNodesNum + 1) +
+            "server nodes=" + serverNodesNum +
+            ", client nodes=" + clientNodesNum +
             ", CPUs=" + totalCpus +
             ", heap=" + heap + "GB" +
             ']';
@@ -2108,6 +2122,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /** Remote nodes. */
         private final List<ClusterNode> rmtNodes;
 
+        /** Client nodes. */
+        private final List<ClusterNode> clientNodes;
+
+        /** Server nodes. */
+        private final List<ClusterNode> serverNodes;
+
         /** All nodes. */
         private final List<ClusterNode> allNodes;
 
@@ -2196,6 +2216,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             all.addAll(rmtNodes);
 
+            clientNodes = Collections.unmodifiableList(new ArrayList<>(F.view(all, clientFilter)));
+
+            serverNodes = Collections.unmodifiableList(new ArrayList<>(F.view(all, F.not(clientFilter))));
+
             Collections.sort(all, GridNodeOrderComparator.INSTANCE);
 
             allNodes = Collections.unmodifiableList(all);
@@ -2346,6 +2370,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             return rmtNodes;
         }
 
+        /** @return Server nodes. */
+        Collection<ClusterNode> serverNodes() {
+            return serverNodes;
+        }
+
+        /** @return Client nodes. */
+        Collection<ClusterNode> clientNodes() {
+            return clientNodes;
+        }
+
         /** @return All nodes. */
         Collection<ClusterNode> allNodes() {
             return allNodes;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8006a843/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
new file mode 100644
index 0000000..efbc431
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.discovery;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.testframework.junits.logger.*;
+import org.apache.log4j.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteTopologyPrintFormatSelfTest extends GridCommonAbstractTest {
+    /** */
+    public static final String TOPOLOGY_SNAPSHOT = "Topology snapshot";
+
+    /** */
+    public static final String SERV_NODE = ">>> Number of server nodes";
+
+    /** */
+    public static final String CLIENT_NODE = ">>> Number of client nodes";
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disc = new TcpDiscoverySpi();
+        disc.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disc);
+
+        if (gridName.endsWith("client"))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        if (log instanceof MockLogger)
+            ((MockLogger)log).clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.INFO);
+
+        doServerLogTest(log);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerDebugLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.DEBUG);
+
+        doServerLogTest(log);
+    }
+
+    /**
+     * @param log Logger.
+     * @throws Exception If failed.
+     */
+    private void doServerLogTest(MockLogger log) throws Exception {
+        try {
+            Ignite server = startGrid("server");
+
+            setLogger(log, server);
+
+            Ignite server1 = startGrid("server1");
+
+            waitForDiscovery(server, server1);
+        }
+        finally {
+            stopAllGrids();
+        }
+
+        assertTrue(F.forAny(log.logs(), new IgnitePredicate<String>() {
+            @Override public boolean apply(String s) {
+                return s.contains("Topology snapshot [ver=2, server nodes=2, client nodes=0,")
+                    || (s.contains(">>> Number of server nodes: 2") && s.contains(">>> Number of client nodes: 0"));
+            }
+        }));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerAndClientLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.INFO);
+
+        doServerAndClientTest(log);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerAndClientDebugLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.DEBUG);
+
+        doServerAndClientTest(log);
+    }
+
+    /**
+     * @param log Log.
+     * @throws Exception If failed.
+     */
+    private void doServerAndClientTest(MockLogger log) throws Exception {
+        try {
+            Ignite server = startGrid("server");
+
+            setLogger(log, server);
+
+            Ignite server1 = startGrid("server1");
+            Ignite client1 = startGrid("first client");
+            Ignite client2 = startGrid("second client");
+
+            waitForDiscovery(server, server1, client1, client2);
+        }
+        finally {
+            stopAllGrids();
+        }
+
+        assertTrue(F.forAny(log.logs(), new IgnitePredicate<String>() {
+            @Override public boolean apply(String s) {
+                return s.contains("Topology snapshot [ver=4, server nodes=2, client nodes=2,")
+                    || (s.contains(">>> Number of server nodes: 2") && s.contains(">>> Number of client nodes: 2"));
+            }
+        }));
+    }
+
+    /**
+     * Set log.
+     *
+     * @param log Log.
+     * @param server Ignite.
+     */
+    private void setLogger(MockLogger log, Ignite server) {
+        IgniteKernal server0 = (IgniteKernal)server;
+
+        GridDiscoveryManager discovery = server0.context().discovery();
+
+        GridTestUtils.setFieldValue(discovery, GridManagerAdapter.class, "log", log);
+    }
+
+    /**
+     *
+     */
+    private static class MockLogger extends GridTestLog4jLogger {
+        /** */
+        private List<String> logs = new ArrayList<>();
+
+        /**  {@inheritDoc} */
+        @Override public void debug(String msg) {
+            if ((msg != null && !msg.isEmpty()) && (
+                msg.contains(TOPOLOGY_SNAPSHOT)
+                    || msg.contains(SERV_NODE)
+                    || msg.contains(CLIENT_NODE)))
+                logs.add(msg);
+
+            super.debug(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void info(String msg) {
+            if ((msg != null && !msg.isEmpty()) && (
+                msg.contains(TOPOLOGY_SNAPSHOT)
+                || msg.contains(SERV_NODE)
+                || msg.contains(CLIENT_NODE)))
+                logs.add(msg);
+
+            super.info(msg);
+        }
+
+        /**
+         * @return Logs.
+         */
+        public List<String> logs() {
+            return logs;
+        }
+
+        /** */
+        public void clear() {
+            logs.clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8006a843/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index e8c89ec..575f1fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -83,6 +83,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridKernalConcurrentAccessStopSelfTest.class);
         suite.addTestSuite(GridUpdateNotifierSelfTest.class);
         suite.addTestSuite(GridLocalEventListenerSelfTest.class);
+        suite.addTestSuite(IgniteTopologyPrintFormatSelfTest.class);
 
         // Managed Services.
         suite.addTestSuite(GridServiceProcessorSingleNodeSelfTest.class);


[29/40] incubator-ignite git commit: # ignite-sprint-7 minor

Posted by sb...@apache.org.
# ignite-sprint-7 minor


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

Branch: refs/heads/ignite-648-failover
Commit: e0c53b8171add7bfed0249c67a8e743443b7593f
Parents: e91bc48
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 1 10:06:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 1 10:06:20 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0c53b81/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index fa3e564..720ce3a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -2332,7 +2332,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 LT.warn(log, null, "Local node has detected failed nodes and started cluster-wide procedure. " +
                         "To speed up failure detection please see 'Failure Detection' section under javadoc" +
-                        "for 'TcpDiscoverySpi'");
+                        " for 'TcpDiscoverySpi'");
             }
         }
 


[02/40] incubator-ignite git commit: IGNITE-1046 Added default option libs.

Posted by sb...@apache.org.
IGNITE-1046 Added default option libs.


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

Branch: refs/heads/ignite-648-failover
Commit: f57aa921e7ac7b583b0f31931890f1f9454f3b1a
Parents: 9de7d24
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri Jun 26 13:06:45 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jun 26 13:06:45 2015 +0300

----------------------------------------------------------------------
 modules/docker/execute.sh |  4 ++++
 modules/docker/run.sh     | 11 ++++++++---
 2 files changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f57aa921/modules/docker/execute.sh
----------------------------------------------------------------------
diff --git a/modules/docker/execute.sh b/modules/docker/execute.sh
index 2aaaa2d..e0ed90c 100644
--- a/modules/docker/execute.sh
+++ b/modules/docker/execute.sh
@@ -8,6 +8,10 @@ if [ ! -z "$GIT_REPO" ]; then
   fi
 fi
 
+if [ -z "$OPTION_LIBS" ]; then
+  OPTION_LIBS="ignite-log4j"
+fi
+
 if [ ! -z "$OPTION_LIBS" ]; then
   IFS=, LIBS_LIST=("$OPTION_LIBS")
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f57aa921/modules/docker/run.sh
----------------------------------------------------------------------
diff --git a/modules/docker/run.sh b/modules/docker/run.sh
index 42fb29d..2e3110d 100644
--- a/modules/docker/run.sh
+++ b/modules/docker/run.sh
@@ -1,11 +1,16 @@
 #!/bin/bash
 
-if [ -z $SKIP_DOWNLOAD ]; then
-  ./download_ignite.sh
-fi
+IGNITE_VERSION=""
 
 if [ -z $SKIP_BUILD_LIBS ]; then
   ./build_users_libs.sh
+
+  IGNITE_VERSION=$(mvn -f user-repo/pom.xml dependency:list | grep ':ignite-core:jar:.*:' | \
+    sed -rn 's/.*([0-9]+\.[0-9]+\.[0-9]+).*/\1/p')
+fi
+
+if [ -z $SKIP_DOWNLOAD ]; then
+  IGNITE_VERSION=$IGNITE_VERSION ./download_ignite.sh
 fi
 
 ./execute.sh
\ No newline at end of file


[03/40] incubator-ignite git commit: IGNITE-1046 Added license.

Posted by sb...@apache.org.
IGNITE-1046 Added license.


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

Branch: refs/heads/ignite-648-failover
Commit: 86c5d207094947aa9e53ea6da6314c125188ec8d
Parents: f57aa92
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri Jun 26 13:09:59 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jun 26 13:09:59 2015 +0300

----------------------------------------------------------------------
 modules/docker/Dockerfile          | 17 +++++++++++++++++
 modules/docker/build_users_libs.sh | 16 ++++++++++++++++
 modules/docker/download_ignite.sh  | 16 ++++++++++++++++
 modules/docker/execute.sh          | 16 ++++++++++++++++
 modules/docker/run.sh              | 16 ++++++++++++++++
 5 files changed, 81 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/86c5d207/modules/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/modules/docker/Dockerfile b/modules/docker/Dockerfile
index 2723333..11e2e3d 100644
--- a/modules/docker/Dockerfile
+++ b/modules/docker/Dockerfile
@@ -1,3 +1,20 @@
+#
+# 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.
+#
+
 # Start from a Debian image.
 FROM debian:8
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/86c5d207/modules/docker/build_users_libs.sh
----------------------------------------------------------------------
diff --git a/modules/docker/build_users_libs.sh b/modules/docker/build_users_libs.sh
index 24f4420..4d37187 100644
--- a/modules/docker/build_users_libs.sh
+++ b/modules/docker/build_users_libs.sh
@@ -1,4 +1,20 @@
 #!/bin/bash
+#
+# 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.
+#
 
 if [ -z $GIT_REPO ]; then
   echo Users git repo is not provided.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/86c5d207/modules/docker/download_ignite.sh
----------------------------------------------------------------------
diff --git a/modules/docker/download_ignite.sh b/modules/docker/download_ignite.sh
index 6a8617f..9e2cb51 100644
--- a/modules/docker/download_ignite.sh
+++ b/modules/docker/download_ignite.sh
@@ -1,4 +1,20 @@
 #!/bin/bash
+#
+# 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.
+#
 
 function download {
   wget -O ignite.zip $1

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/86c5d207/modules/docker/execute.sh
----------------------------------------------------------------------
diff --git a/modules/docker/execute.sh b/modules/docker/execute.sh
index e0ed90c..8afba59 100644
--- a/modules/docker/execute.sh
+++ b/modules/docker/execute.sh
@@ -1,4 +1,20 @@
 #!/bin/bash
+#
+# 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.
+#
 
 if [ ! -z "$GIT_REPO" ]; then
   if [ -z "$LIB_PATTERN" ]; then

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/86c5d207/modules/docker/run.sh
----------------------------------------------------------------------
diff --git a/modules/docker/run.sh b/modules/docker/run.sh
index 2e3110d..e7feb8c 100644
--- a/modules/docker/run.sh
+++ b/modules/docker/run.sh
@@ -1,4 +1,20 @@
 #!/bin/bash
+#
+# 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.
+#
 
 IGNITE_VERSION=""
 


[31/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-7' into ignite-sprint-7

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


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

Branch: refs/heads/ignite-648-failover
Commit: bccba308a884ef527cf7f84f7f6dd57d55b1e51a
Parents: aea804b e0c53b8
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jul 1 11:13:36 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jul 1 11:13:36 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[35/40] incubator-ignite git commit: IGNITE-1059: Fixed (thanks to Atri for help).

Posted by sb...@apache.org.
IGNITE-1059: Fixed (thanks to Atri for help).


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

Branch: refs/heads/ignite-648-failover
Commit: 5269baba23e062e19981cd4c09d6f2271925fffd
Parents: 204823c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jul 1 16:35:38 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jul 1 16:35:38 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheFutureImpl.java | 42 ++++++++++++++++++++
 .../processors/cache/IgniteCacheProxy.java      |  2 +-
 .../internal/util/future/IgniteFutureImpl.java  | 18 +++++++--
 3 files changed, 57 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5269baba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
new file mode 100644
index 0000000..06c28e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
@@ -0,0 +1,42 @@
+/*
+ * 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.internal.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+/**
+ * Implementation of public API future for cache.
+ */
+public class IgniteCacheFutureImpl<V> extends IgniteFutureImpl<V> {
+    /**
+     * Constructor.
+     *
+     * @param fut Internal future.
+     */
+    public IgniteCacheFutureImpl(IgniteInternalFuture<V> fut) {
+        super(fut);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RuntimeException convertException(IgniteCheckedException e) {
+        return CU.convertToCacheException(e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5269baba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 48fd259..3360727 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -1555,7 +1555,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
      * @param fut Future for async operation.
      */
     private <R> void setFuture(IgniteInternalFuture<R> fut) {
-        curFut.set(new IgniteFutureImpl<>(fut));
+        curFut.set(new IgniteCacheFutureImpl<>(fut));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5269baba/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java
index 86ad438..764e0ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/IgniteFutureImpl.java
@@ -100,7 +100,7 @@ public class IgniteFutureImpl<V> implements IgniteFuture<V> {
             return fut.cancel();
         }
         catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+            throw convertException(e);
         }
     }
 
@@ -110,7 +110,7 @@ public class IgniteFutureImpl<V> implements IgniteFuture<V> {
             return fut.get();
         }
         catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+            throw convertException(e);
         }
     }
 
@@ -120,7 +120,7 @@ public class IgniteFutureImpl<V> implements IgniteFuture<V> {
             return fut.get(timeout);
         }
         catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+            throw convertException(e);
         }
     }
 
@@ -130,10 +130,20 @@ public class IgniteFutureImpl<V> implements IgniteFuture<V> {
             return fut.get(timeout, unit);
         }
         catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+            throw convertException(e);
         }
     }
 
+    /**
+     * Convert internal exception to public exception.
+     *
+     * @param e Internal exception.
+     * @return Public excpetion.
+     */
+    protected RuntimeException convertException(IgniteCheckedException e) {
+        return U.convertException(e);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return "IgniteFuture [orig=" + fut + ']';


[32/40] incubator-ignite git commit: IGNITE-1068: Fixing ordered messages back pressure control.

Posted by sb...@apache.org.
IGNITE-1068: Fixing ordered messages back pressure control.


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

Branch: refs/heads/ignite-648-failover
Commit: c7e7487458b72ba8b666fba7b9e24004d425719f
Parents: bccba30
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jul 1 11:50:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jul 1 11:50:08 2015 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 49 +++++++++++---------
 .../util/nio/GridNioMessageTracker.java         | 23 +++++++--
 2 files changed, 48 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7e74874/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 4382731..d8dcc2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -718,7 +719,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             map = msgSetMap.get(msg.topic());
 
             if (map == null) {
-                set = new GridCommunicationMessageSet(plc, msg.topic(), nodeId, timeout, skipOnTimeout, msg);
+                set = new GridCommunicationMessageSet(plc, msg.topic(), nodeId, timeout, skipOnTimeout, msg, msgC);
 
                 map = new ConcurrentHashMap0<>();
 
@@ -748,7 +749,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                     if (set == null) {
                         GridCommunicationMessageSet old = map.putIfAbsent(nodeId,
                             set = new GridCommunicationMessageSet(plc, msg.topic(),
-                                nodeId, timeout, skipOnTimeout, msg));
+                                nodeId, timeout, skipOnTimeout, msg, msgC));
 
                         assert old == null;
 
@@ -766,7 +767,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 assert set != null;
                 assert !isNew;
 
-                set.add(msg);
+                set.add(msg, msgC);
 
                 break;
             }
@@ -795,14 +796,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         if (isNew && set.endTime() != Long.MAX_VALUE)
             ctx.timeout().addTimeoutObject(set);
 
-        if (set.reserved()) {
-            // Set is reserved which means that it is currently processed by worker thread.
-            if (msgC != null)
-                msgC.run();
-
-            return;
-        }
-
         final GridMessageListener lsnr = lsnrMap.get(msg.topic());
 
         if (lsnr == null) {
@@ -821,7 +814,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                     "listener is registered): " + msg);
             }
 
-            // Mark the message as processed.
+            // Mark the message as processed, otherwise reading from the connection
+            // may stop.
             if (msgC != null)
                 msgC.run();
 
@@ -848,8 +842,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 }
                 finally {
                     threadProcessingMessage(false);
-
-                    msgC.run();
                 }
             }
         };
@@ -1852,7 +1844,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
         /** */
         @GridToStringInclude
-        private final Queue<IgniteBiTuple<GridIoMessage, Long>> msgs = new ConcurrentLinkedDeque<>();
+        private final Queue<GridTuple3<GridIoMessage, Long, IgniteRunnable>> msgs = new ConcurrentLinkedDeque<>();
 
         /** */
         private final AtomicBoolean reserved = new AtomicBoolean();
@@ -1873,6 +1865,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
          * @param timeout Timeout.
          * @param skipOnTimeout Whether message can be skipped on timeout.
          * @param msg Message to add immediately.
+         * @param msgC Message closure (may be {@code null}).
          */
         GridCommunicationMessageSet(
             GridIoPolicy plc,
@@ -1880,7 +1873,8 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             UUID nodeId,
             long timeout,
             boolean skipOnTimeout,
-            GridIoMessage msg
+            GridIoMessage msg,
+            @Nullable IgniteRunnable msgC
         ) {
             assert nodeId != null;
             assert topic != null;
@@ -1899,7 +1893,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
             lastTs = U.currentTimeMillis();
 
-            msgs.add(F.t(msg, lastTs));
+            msgs.add(F.t(msg, lastTs, msgC));
         }
 
         /** {@inheritDoc} */
@@ -2017,15 +2011,28 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         void unwind(GridMessageListener lsnr) {
             assert reserved.get();
 
-            for (IgniteBiTuple<GridIoMessage, Long> t = msgs.poll(); t != null; t = msgs.poll())
-                lsnr.onMessage(nodeId, t.get1().message());
+            for (GridTuple3<GridIoMessage, Long, IgniteRunnable> t = msgs.poll(); t != null; t = msgs.poll()) {
+                try {
+                    lsnr.onMessage(
+                        nodeId,
+                        t.get1().message());
+                }
+                finally {
+                    if (t.get3() != null)
+                        t.get3().run();
+                }
+            }
         }
 
         /**
          * @param msg Message to add.
+         * @param msgC Message closure (may be {@code null}).
          */
-        void add(GridIoMessage msg) {
-            msgs.add(F.t(msg, U.currentTimeMillis()));
+        void add(
+            GridIoMessage msg,
+            @Nullable IgniteRunnable msgC
+        ) {
+            msgs.add(F.t(msg, U.currentTimeMillis(), msgC));
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7e74874/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java
index 52b7fed..c9ed1a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioMessageTracker.java
@@ -56,9 +56,26 @@ public class GridNioMessageTracker implements IgniteRunnable {
 
     /** {@inheritDoc} */
     @Override public void run() {
-        int cnt = msgCnt.decrementAndGet();
+        // In case of ordered messages this may be called twice for 1 message.
+        // Example: message arrives, but listener has not been installed yet.
+        // Message set is created, but message does not get actually processed.
+        // If this is not called, connection may be paused which causes hang.
+        // It seems acceptable to have the following logic accounting the aforementioned.
+        int cnt = 0;
 
-        assert cnt >= 0 : "Invalid count: " + cnt;
+        for (;;) {
+            int cur = msgCnt.get();
+
+            if (cur == 0)
+                break;
+
+            cnt = cur - 1;
+
+            if (msgCnt.compareAndSet(cur, cnt))
+                break;
+        }
+
+        assert cnt >= 0 : "Invalid count [cnt=" + cnt + ", this=" + this + ']';
 
         if (cnt < msgQueueLimit && paused && lock.tryLock()) {
             try {
@@ -116,6 +133,6 @@ public class GridNioMessageTracker implements IgniteRunnable {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridNioMessageTracker.class, this, super.toString());
+        return S.toString(GridNioMessageTracker.class, this, "hash", System.identityHashCode(this));
     }
 }


[30/40] incubator-ignite git commit: Merge branch 'ignite-sprint-7' into ignite-1046

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-7' into ignite-1046


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

Branch: refs/heads/ignite-648-failover
Commit: aea804b41fd0a73a1eb4f8f68e28bf77d00de966
Parents: 4eef6d0 e91bc48
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jul 1 11:03:32 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jul 1 11:03:32 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractConnectivitySelfTest.java     |   4 +-
 .../org/apache/ignite/cluster/ClusterGroup.java |  18 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   2 +
 .../ignite/compute/ComputeTaskSplitAdapter.java |   2 +-
 .../configuration/CacheConfiguration.java       | 105 +--
 .../configuration/IgniteReflectionFactory.java  |  81 +-
 .../configuration/NearCacheConfiguration.java   |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |   5 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  50 +-
 .../cluster/IgniteClusterAsyncImpl.java         |  12 +-
 .../discovery/GridDiscoveryManager.java         |  32 +-
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../processors/cache/GridCacheProcessor.java    |  62 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   9 +-
 .../GridDhtPartitionsExchangeFuture.java        |  46 +-
 .../processors/clock/GridClockServer.java       |  21 +-
 .../datastructures/DataStructuresProcessor.java |  64 +-
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 .../processors/plugin/CachePluginManager.java   |  10 +-
 .../processors/rest/GridRestProcessor.java      |   4 +-
 .../handlers/task/GridTaskCommandHandler.java   |  12 +-
 .../processors/task/GridTaskWorker.java         |   4 +-
 .../internal/util/GridConfigurationFinder.java  |  55 +-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../apache/ignite/internal/visor/VisorJob.java  |   2 +
 .../internal/visor/log/VisorLogSearchTask.java  |   2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   4 +
 .../visor/query/VisorQueryCleanupTask.java      |  14 +
 .../util/VisorClusterGroupEmptyException.java   |  37 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 151 ++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 103 ++-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   3 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  74 +-
 .../core/src/test/config/spark/spark-config.xml |  46 ++
 .../internal/ClusterGroupAbstractTest.java      | 777 ++++++++++++++++++
 .../internal/ClusterGroupHostsSelfTest.java     | 141 ++++
 .../ignite/internal/ClusterGroupSelfTest.java   | 251 ++++++
 .../internal/GridDiscoveryEventSelfTest.java    |  12 +-
 .../internal/GridProjectionAbstractTest.java    | 784 -------------------
 .../ignite/internal/GridProjectionSelfTest.java | 251 ------
 .../apache/ignite/internal/GridSelfTest.java    |   2 +-
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 +++++++
 .../CacheReadThroughAtomicRestartSelfTest.java  |  32 +
 ...heReadThroughLocalAtomicRestartSelfTest.java |  32 +
 .../CacheReadThroughLocalRestartSelfTest.java   |  32 +
 ...dThroughReplicatedAtomicRestartSelfTest.java |  32 +
 ...cheReadThroughReplicatedRestartSelfTest.java |  32 +
 .../cache/CacheReadThroughRestartSelfTest.java  | 133 ++++
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ---
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  37 +
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   4 +-
 .../GridCacheSetAbstractSelfTest.java           |  22 +-
 .../IgniteDataStructureWithJobTest.java         | 111 +++
 .../distributed/IgniteCache150ClientsTest.java  | 189 +++++
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../distributed/IgniteCacheManyClientsTest.java |   1 +
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   5 -
 ...achePartitionedMultiNodeFullApiSelfTest.java |  53 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 +
 .../IgniteCacheTxStoreSessionTest.java          |   4 +
 .../internal/util/IgniteUtilsSelfTest.java      |  22 +
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 265 ++++++-
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   7 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../testsuites/IgniteCacheTestSuite4.java       |   8 +
 .../testsuites/IgniteClientTestSuite.java       |  38 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 .../p2p/GridP2PContinuousDeploymentTask1.java   |   2 +-
 modules/hadoop/pom.xml                          |  78 --
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 +
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +-
 .../internal/processors/hadoop/HadoopUtils.java | 237 ------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   | 241 ++++++
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 .../ignite/schema/model/PojoDescriptor.java     |   2 +
 .../apache/ignite/schema/model/PojoField.java   |   1 +
 .../parser/dialect/OracleMetadataDialect.java   |   2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |  50 +-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 +
 .../commands/cache/VisorCacheCommand.scala      |   7 +-
 pom.xml                                         |  12 +-
 scripts/git-patch-prop.sh                       |   2 +-
 109 files changed, 4514 insertions(+), 1876 deletions(-)
----------------------------------------------------------------------



[18/40] incubator-ignite git commit: # sprint-7 Minor fixes in javadocs.

Posted by sb...@apache.org.
# sprint-7 Minor fixes in javadocs.


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

Branch: refs/heads/ignite-648-failover
Commit: 596c5389b0faf1de1be5cbe2a88b5579a93683b2
Parents: 1a21477
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 30 10:26:12 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 30 10:26:12 2015 +0700

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/schema/model/PojoDescriptor.java   | 2 ++
 .../src/main/java/org/apache/ignite/schema/model/PojoField.java    | 1 +
 .../apache/ignite/schema/parser/dialect/OracleMetadataDialect.java | 2 +-
 3 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/596c5389/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
index 4ef8b00..5e1da06 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
@@ -304,6 +304,8 @@ public class PojoDescriptor {
 
     /**
      * Gets indexes groups.
+     *
+     * @return Map with indexes.
      */
     public Map<String, Map<String, IndexItem>> groups() {
         Map<String, Map<String, Boolean>> idxs = tbl.indexes();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/596c5389/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoField.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoField.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoField.java
index 10939d9..88e4008 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoField.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoField.java
@@ -184,6 +184,7 @@ public class PojoField {
 
     /**
      * @param jdbcType String name for JDBC type.
+     * @return String name for JDBC type.
      */
     private static String jdbcTypeName(int jdbcType) {
         switch (jdbcType) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/596c5389/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
index 6b16042..855c9f7 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
@@ -189,7 +189,7 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
      * @param owner DB owner.
      * @param tbl Table name.
      * @return Index columns.
-     * @throws SQLException If failed to retrieve indexe columns.
+     * @throws SQLException If failed to retrieve indexes columns.
      */
     private Map<String, Map<String, Boolean>> indexes(PreparedStatement stmt, String owner, String tbl)
         throws SQLException {


[17/40] incubator-ignite git commit: # ignite-648: the rest of tests

Posted by sb...@apache.org.
# ignite-648: the rest of tests


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

Branch: refs/heads/ignite-648-failover
Commit: aaca0e8b3662dbcfbf5f514abb830039cf961803
Parents: 3c7ff71
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 29 22:24:20 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 29 22:24:20 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAbstractFullApiSelfTest.java      | 3 +++
 .../near/GridCachePartitionedMultiNodeFullApiSelfTest.java      | 5 +++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aaca0e8b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index d295be6..9b9c86a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -4723,6 +4723,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testWithSkipStoreRemoveAll() throws Exception {
+        if (isMultiJvm())
+            return;
+
         if (atomicityMode() == TRANSACTIONAL || (atomicityMode() == ATOMIC && nearEnabled())) // TODO IGNITE-373.
             return;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aaca0e8b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 003c186..0232efe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -327,7 +327,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         for (int i = 0; i < gridCount(); i++) {
             IgniteEx ignite = grid(i);
 
-            if (!ignite.configuration().isClientMode()) {
+            if (!Boolean.TRUE.equals(ignite.configuration().isClientMode())) {
                 if (ignite0 == null)
                     ignite0 = ignite;
                 else if (ignite1 == null)
@@ -417,7 +417,8 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
             }
         };
 
-        info("All affinity nodes: " + affinityNodes());
+        if (!isMultiJvm())
+            info("All affinity nodes: " + affinityNodes());
 
         IgniteCache<Object, Object> cache = grid(0).cache(null);
 


[26/40] incubator-ignite git commit: IGNITE-1062 Added tests.

Posted by sb...@apache.org.
IGNITE-1062 Added tests.


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

Branch: refs/heads/ignite-648-failover
Commit: 6ebcb6de192e1916e58226990eb9d9ec759da6ef
Parents: 8006a84
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 30 16:51:15 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 30 16:51:15 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         | 26 +--------
 .../IgniteTopologyPrintFormatSelfTest.java      | 60 +++++++++++++++++++-
 2 files changed, 61 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6ebcb6de/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 1d555e4..a8ce8ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -101,7 +101,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** Predicate filtering client nodes. */
     private static final IgnitePredicate<ClusterNode> clientFilter = new P1<ClusterNode>() {
         @Override public boolean apply(ClusterNode n) {
-            return n.isClient();
+            return CU.clientNode(n);
         }
     };
 
@@ -940,9 +940,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         Collection<ClusterNode> rmtNodes = discoCache.remoteNodes();
 
-        Collection<ClusterNode> serverNodes = discoCache.serverNodes();
+        Collection<ClusterNode> serverNodes = F.view(discoCache.allNodes(), F.not(clientFilter));
 
-        Collection<ClusterNode> clientNodes = discoCache.clientNodes();
+        Collection<ClusterNode> clientNodes = F.view(discoCache.allNodes(), clientFilter);
 
         ClusterNode locNode = discoCache.localNode();
 
@@ -2122,12 +2122,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /** Remote nodes. */
         private final List<ClusterNode> rmtNodes;
 
-        /** Client nodes. */
-        private final List<ClusterNode> clientNodes;
-
-        /** Server nodes. */
-        private final List<ClusterNode> serverNodes;
-
         /** All nodes. */
         private final List<ClusterNode> allNodes;
 
@@ -2216,10 +2210,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             all.addAll(rmtNodes);
 
-            clientNodes = Collections.unmodifiableList(new ArrayList<>(F.view(all, clientFilter)));
-
-            serverNodes = Collections.unmodifiableList(new ArrayList<>(F.view(all, F.not(clientFilter))));
-
             Collections.sort(all, GridNodeOrderComparator.INSTANCE);
 
             allNodes = Collections.unmodifiableList(all);
@@ -2370,16 +2360,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             return rmtNodes;
         }
 
-        /** @return Server nodes. */
-        Collection<ClusterNode> serverNodes() {
-            return serverNodes;
-        }
-
-        /** @return Client nodes. */
-        Collection<ClusterNode> clientNodes() {
-            return clientNodes;
-        }
-
         /** @return All nodes. */
         Collection<ClusterNode> allNodes() {
             return allNodes;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6ebcb6de/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
index efbc431..2a71f28 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
@@ -56,11 +56,16 @@ public class IgniteTopologyPrintFormatSelfTest extends GridCommonAbstractTest {
         TcpDiscoverySpi disc = new TcpDiscoverySpi();
         disc.setIpFinder(IP_FINDER);
 
-        cfg.setDiscoverySpi(disc);
-
         if (gridName.endsWith("client"))
             cfg.setClientMode(true);
 
+        if (gridName.endsWith("client_force_server")) {
+            cfg.setClientMode(true);
+            disc.setForceServerMode(true);
+        }
+
+        cfg.setDiscoverySpi(disc);
+
         return cfg;
     }
 
@@ -176,6 +181,57 @@ public class IgniteTopologyPrintFormatSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testForceServerAndClientLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.INFO);
+
+        doForceServerAndClientTest(log);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testForceServerAndClientDebugLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.DEBUG);
+
+        doForceServerAndClientTest(log);
+    }
+
+    /**
+     * @param log Log.
+     * @throws Exception If failed.
+     */
+    private void doForceServerAndClientTest(MockLogger log) throws Exception {
+        try {
+            Ignite server = startGrid("server");
+
+            setLogger(log, server);
+
+            Ignite server1 = startGrid("server1");
+            Ignite client1 = startGrid("first client");
+            Ignite client2 = startGrid("second client");
+            Ignite forceServClnt3 = startGrid("third client_force_server");
+
+            waitForDiscovery(server, server1, client1, client2, forceServClnt3);
+        }
+        finally {
+            stopAllGrids();
+        }
+
+        assertTrue(F.forAny(log.logs(), new IgnitePredicate<String>() {
+            @Override public boolean apply(String s) {
+                return s.contains("Topology snapshot [ver=5, server nodes=2, client nodes=3,")
+                    || (s.contains(">>> Number of server nodes: 2") && s.contains(">>> Number of client nodes: 3"));
+            }
+        }));
+    }
+
+    /**
      * Set log.
      *
      * @param log Log.


[11/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-7' into ignite-sprint-7

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


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

Branch: refs/heads/ignite-648-failover
Commit: 5dff89c650575b6fe92450492027e77ffce48186
Parents: a226666 15f3612
Author: avinogradov <av...@gridgain.com>
Authored: Mon Jun 29 15:56:02 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Jun 29 15:56:02 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++++--
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ------------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 6 files changed, 245 insertions(+), 132 deletions(-)
----------------------------------------------------------------------



[34/40] incubator-ignite git commit: # ignite-648: IgniteClusterProcessProxy.java instead of local cluster

Posted by sb...@apache.org.
# ignite-648: IgniteClusterProcessProxy.java instead of local cluster


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

Branch: refs/heads/ignite-648-failover
Commit: ae4e791fd5d31d1cad229510dd3337860eaf2959
Parents: aaca0e8
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 15:20:10 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 15:20:10 2015 +0300

----------------------------------------------------------------------
 .../multijvm/IgniteClusterProcessProxy.java     | 320 +++++++++++++++++++
 .../multijvm/IgniteEventsProcessProxy.java      |  32 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  94 +++---
 3 files changed, 383 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae4e791f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
new file mode 100644
index 0000000..99ec424
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
@@ -0,0 +1,320 @@
+/*
+ * 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.testframework.junits.multijvm;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Proxy class for cluster at another jvm.
+ */
+@SuppressWarnings("TransientFieldInNonSerializableClass")
+public class IgniteClusterProcessProxy implements IgniteClusterEx {
+    /** Grid id. */
+    private final UUID gridId;
+
+    /** Compute. */
+    private final transient IgniteCompute compute;
+
+    /** */
+    private final IgniteProcessProxy proxy;
+
+    /**
+     * @param proxy Ignite Proxy.
+     */
+    public IgniteClusterProcessProxy(IgniteProcessProxy proxy) {
+        this.proxy = proxy;
+        gridId = proxy.getId();
+        compute = proxy.remoteCompute();
+    }
+
+    /**
+     * Returns cluster instance. Method to be called from closure at another JVM.
+     *
+     * @return Cache.
+     */
+    private IgniteClusterEx cluster() {
+        return (IgniteClusterEx)Ignition.ignite(gridId).cluster();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroupEx forSubjectId(final UUID subjId) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forCacheNodes(@Nullable String cacheName, boolean affNodes, boolean nearNodes,
+        boolean clientNodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode localNode() {
+        return compute.call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return cluster().localNode();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forLocal() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> ConcurrentMap<K, V> nodeLocalMap() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public long topologyVersion() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> topology(long topVer) throws UnsupportedOperationException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> Map<ClusterNode, Collection<K>> mapKeysToNodes(@Nullable String cacheName,
+        @Nullable Collection<? extends K> keys) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> ClusterNode mapKeyToNode(@Nullable String cacheName, K key) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterStartNodeResult> startNodes(File file, boolean restart, int timeout,
+        int maxConn) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterStartNodeResult> startNodes(Collection<Map<String, Object>> hosts,
+        @Nullable Map<String, Object> dflts, boolean restart, int timeout, int maxConn) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopNodes() throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stopNodes(Collection<UUID> ids) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restartNodes() throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void restartNodes(Collection<UUID> ids) throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void resetMetrics() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCluster withAsync() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> IgniteFuture<R> future() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Ignite ignite() {
+        return proxy;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNodes(Collection<? extends ClusterNode> nodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNode(ClusterNode node, ClusterNode... nodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forOthers(ClusterNode node, ClusterNode... nodes) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forOthers(ClusterGroup prj) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNodeIds(Collection<UUID> ids) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forNodeId(UUID id, UUID... ids) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forPredicate(IgnitePredicate<ClusterNode> p) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forAttribute(String name, @Nullable Object val) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forServers() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forClients() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forCacheNodes(String cacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forDataNodes(String cacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forClientNodes(String cacheName) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forRemotes() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forHost(ClusterNode node) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forHost(String host, String... hosts) {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forDaemons() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forRandom() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forOldest() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterGroup forYoungest() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> nodes() {
+        return compute.call(new IgniteCallable<Collection<ClusterNode>>() {
+            @Override public Collection<ClusterNode> call() throws Exception {
+                return cluster().nodes();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode node(final UUID nid) {
+        return compute.call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return cluster().node(nid);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode node() {
+        return compute.call(new IgniteCallable<ClusterNode>() {
+            @Override public ClusterNode call() throws Exception {
+                return cluster().node();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> hostNames() {
+        return compute.call(new IgniteCallable<Collection<String>>() {
+            @Override public Collection<String> call() throws Exception {
+                return cluster().hostNames();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgnitePredicate<ClusterNode> predicate() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterMetrics metrics() throws IgniteException {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae4e791f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
index eb09965..018aa8d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteEventsProcessProxy.java
@@ -54,47 +54,47 @@ public class IgniteEventsProcessProxy implements IgniteEvents {
 
     /** {@inheritDoc} */
     @Override public ClusterGroup clusterGroup() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> List<T> remoteQuery(IgnitePredicate<T> p, long timeout,
         @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> UUID remoteListen(@Nullable IgniteBiPredicate<UUID, T> locLsnr,
         @Nullable IgnitePredicate<T> rmtFilter, @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> UUID remoteListen(int bufSize, long interval, boolean autoUnsubscribe,
         @Nullable IgniteBiPredicate<UUID, T> locLsnr, @Nullable IgnitePredicate<T> rmtFilter,
         @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void stopRemoteListen(UUID opId) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> T waitForLocal(@Nullable IgnitePredicate<T> filter,
         @Nullable int... types) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends Event> Collection<T> localQuery(IgnitePredicate<T> p, @Nullable int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void recordLocal(Event evt) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -108,41 +108,41 @@ public class IgniteEventsProcessProxy implements IgniteEvents {
 
     /** {@inheritDoc} */
     @Override public boolean stopLocalListen(IgnitePredicate<? extends Event> lsnr, @Nullable int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void enableLocal(int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public void disableLocal(int... types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public int[] enabledEvents() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isEnabled(int type) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteEvents withAsync() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isAsync() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <R> IgniteFuture<R> future() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ae4e791f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 5ed5b87..b96fe5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -77,7 +77,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
         List<String> jvmArgs = U.jvmArgs();
 
-        List<String> filteredJvmArgs = new ArrayList<>();
+        Collection<String> filteredJvmArgs = new ArrayList<>();
 
         for (String arg : jvmArgs) {
             if(!arg.toLowerCase().startsWith("-agentlib"))
@@ -195,63 +195,63 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K extends GridCacheUtilityKey, V> IgniteInternalCache<K, V> utilityCache() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex(@Nullable String name) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public <K, V> IgniteInternalCache<K, V> cachex() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteInternalCache<?, ?>> cachesx(
         @Nullable IgnitePredicate<? super IgniteInternalCache<?, ?>>... p) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean eventUserRecordable(int type) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean allEventsUserRecordable(int[] types) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isJmxRemoteEnabled() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public boolean isRestartEnabled() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public IgniteFileSystem igfsx(@Nullable String name) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Hadoop hadoop() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteClusterEx cluster() {
-        return (IgniteClusterEx)locJvmGrid.cluster();
+        return new IgniteClusterProcessProxy(this);
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public String latestVersion() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -265,27 +265,27 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public GridKernalContext context() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCompute compute() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCompute compute(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteMessaging message() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteMessaging message(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -295,85 +295,85 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public IgniteEvents events(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteServices services() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteServices services(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public ExecutorService executorService() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public ExecutorService executorService(ClusterGroup grp) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteProductVersion version() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteScheduler scheduler() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName,
         NearCacheConfiguration<K, V> nearCfg) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -383,7 +383,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable final String name) {
-        return new IgniteCacheProcessProxy(name, this);
+        return new IgniteCacheProcessProxy<>(name, this);
     }
 
     /** {@inheritDoc} */
@@ -393,61 +393,61 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFileSystem fileSystem(String name) {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFileSystem> fileSystems() {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteAtomicReference<T> atomicReference(String name, @Nullable T initVal,
         boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override  public <T, S> IgniteAtomicStamped<T, S> atomicStamped(String name, @Nullable T initVal, @Nullable S initStamp,
         boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public IgniteCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
         boolean create) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteQueue<T> queue(String name, int cap,
         @Nullable CollectionConfiguration cfg) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteSet<T> set(String name, @Nullable CollectionConfiguration cfg) throws IgniteException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
     @Override public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException {
-        throw new UnsupportedOperationException("Operation doesn't supported yet.");
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 
     /** {@inheritDoc} */
@@ -462,7 +462,7 @@ public class IgniteProcessProxy implements IgniteEx {
 
     /** {@inheritDoc} */
     @Override public <K> Affinity<K> affinity(String cacheName) {
-        return new AffinityProcessProxy(cacheName, this);
+        return new AffinityProcessProxy<>(cacheName, this);
     }
 
     /**


[40/40] incubator-ignite git commit: Merge branch 'ignite-648' into ignite-648-failover

Posted by sb...@apache.org.
Merge branch 'ignite-648' into ignite-648-failover


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

Branch: refs/heads/ignite-648-failover
Commit: 430c17e2c289127dc9cf4061ab0198769da715f8
Parents: 4889b6d efb8239
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 17:50:01 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 17:50:01 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 +-
 .../ignite/compute/ComputeTaskSplitAdapter.java |   2 +-
 .../configuration/CacheConfiguration.java       | 105 +++---
 .../configuration/NearCacheConfiguration.java   |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../managers/communication/GridIoManager.java   |  49 +--
 .../discovery/GridDiscoveryManager.java         |  32 +-
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../processors/cache/GridCacheProcessor.java    |  62 +++-
 .../processors/cache/IgniteCacheFutureImpl.java |  42 +++
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../processors/clock/GridClockServer.java       |  21 +-
 .../internal/util/future/IgniteFutureImpl.java  |  18 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../util/nio/GridNioMessageTracker.java         |  23 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  74 ++++-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/spark/spark-config.xml |  46 +++
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 +++++++++++++++++
 .../cache/GridCacheAbstractFullApiSelfTest.java |  71 +++-
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 -------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++
 ...achePartitionedMultiNodeFullApiSelfTest.java |  44 ++-
 ...heAtomicNearOnlyMultiJvmFullApiSelfTest.java |   2 +-
 .../GridP2PContinuousDeploymentSelfTest.java    |   2 -
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../junits/common/GridCommonAbstractTest.java   |  57 +++-
 .../multijvm/IgniteCacheProcessProxy.java       |   4 +-
 .../multijvm/IgniteClusterProcessProxy.java     | 320 +++++++++++++++++++
 .../multijvm/IgniteEventsProcessProxy.java      |  32 +-
 .../junits/multijvm/IgniteProcessProxy.java     | 199 ++++++++----
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 modules/docker/Dockerfile                       |  55 ++++
 modules/docker/README.txt                       |  11 +
 modules/docker/build_users_libs.sh              |  39 +++
 modules/docker/download_ignite.sh               |  49 +++
 modules/docker/execute.sh                       |  62 ++++
 modules/docker/run.sh                           |  34 ++
 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/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 .../ignite/schema/model/PojoDescriptor.java     |   2 +
 .../apache/ignite/schema/model/PojoField.java   |   1 +
 .../parser/dialect/OracleMetadataDialect.java   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |  50 ++-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 ++
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  14 +-
 82 files changed, 1845 insertions(+), 412 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/430c17e2/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/430c17e2/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------


[23/40] incubator-ignite git commit: # ignite-gg-1064 allow start if can not initialize multicast

Posted by sb...@apache.org.
# ignite-gg-1064 allow start if can not initialize multicast


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

Branch: refs/heads/ignite-648-failover
Commit: b5bc06ee813471643c460aaa74ef4395ad79bd53
Parents: bade9f1
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 30 12:40:53 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 30 12:40:53 2015 +0300

----------------------------------------------------------------------
 .../TcpDiscoveryMulticastIpFinder.java          | 74 ++++++++++++++++----
 1 file changed, 60 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b5bc06ee/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index 8e5a1fd..014d937 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -26,7 +26,6 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.jetbrains.annotations.*;
@@ -278,7 +277,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
         addrSnds = new ArrayList<>(locAddrs.size());
 
-        Collection<InetAddress> reqItfs = new ArrayList<>(locAddrs.size()); // Interfaces used to send requests.
+        Set<InetAddress> reqItfs = new HashSet<>(locAddrs.size()); // Interfaces used to send requests.
 
         for (String locAddr : locAddrs) {
             InetAddress addr;
@@ -309,6 +308,8 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
             }
         }
 
+        boolean mcastErr = false;
+
         if (!clientMode) {
             if (addrSnds.isEmpty()) {
                 try {
@@ -317,13 +318,31 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                     addrSnds.add(new AddressSender(mcastAddr, null, addrs));
                 }
                 catch (IOException e) {
-                    throw new IgniteSpiException("Failed to create multicast socket [mcastAddr=" + mcastAddr +
-                        ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ']', e);
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to create multicast socket [mcastAddr=" + mcastAddr +
+                            ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ", err=" + e + ']');
+                }
+
+                if (addrSnds.isEmpty()) {
+                    try {
+                        addrSnds.add(new AddressSender(mcastAddr, mcastAddr, addrs));
+
+                        reqItfs.add(mcastAddr);
+                    }
+                    catch (IOException e) {
+                        log.debug("Failed to create multicast socket [mcastAddr=" + mcastAddr +
+                            ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ", locAddr=" + mcastAddr +
+                            ", err=" + e + ']');
+                    }
                 }
             }
 
-            for (AddressSender addrSnd : addrSnds)
-                addrSnd.start();
+            if (!addrSnds.isEmpty()) {
+                for (AddressSender addrSnd : addrSnds)
+                    addrSnd.start();
+            }
+            else
+                mcastErr = true;
         }
         else
             assert addrSnds.isEmpty() : addrSnds;
@@ -358,10 +377,30 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                 }
             }
         }
-        else
-            ret = requestAddresses(mcastAddr, F.first(reqItfs));
+        else {
+            T2<Collection<InetSocketAddress>, Boolean> res = requestAddresses(mcastAddr, F.first(reqItfs));
+
+            ret = res.get1();
+
+            mcastErr |= res.get2();
+        }
+
+        if (ret.isEmpty()) {
+            if (mcastErr) {
+                if (getRegisteredAddresses().isEmpty()) {
+                    InetSocketAddress addr = new InetSocketAddress("localhost", TcpDiscoverySpi.DFLT_PORT);
+
+                    U.quietAndWarn(log, "TcpDiscoveryMulticastIpFinder failed to initialize multicast, " +
+                        "will use default address: " + addr);
 
-        if (!ret.isEmpty())
+                    registerAddresses(Collections.singleton(addr));
+                }
+                else
+                    U.quietAndWarn(log, "TcpDiscoveryMulticastIpFinder failed to initialize multicast, " +
+                        "will use pre-configured addresses.");
+            }
+        }
+        else
             registerAddresses(ret);
     }
 
@@ -379,11 +418,16 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
      *
      * @param mcastAddr Multicast address where to send request.
      * @param sockItf Optional interface multicast socket should be bound to.
-     * @return Collection of received addresses.
+     * @return Tuple where first value is collection of received addresses, second is boolean which is
+     *      {@code true} if got error on send.
      */
-    private Collection<InetSocketAddress> requestAddresses(InetAddress mcastAddr, @Nullable InetAddress sockItf) {
+    private T2<Collection<InetSocketAddress>, Boolean> requestAddresses(InetAddress mcastAddr,
+        @Nullable InetAddress sockItf)
+    {
         Collection<InetSocketAddress> rmtAddrs = new HashSet<>();
 
+        boolean sndErr = false;
+
         try {
             DatagramPacket reqPckt = new DatagramPacket(MSG_ADDR_REQ_DATA, MSG_ADDR_REQ_DATA.length,
                 mcastAddr, mcastPort);
@@ -414,6 +458,8 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                         sock.send(reqPckt);
                     }
                     catch (IOException e) {
+                        sndErr = true;
+
                         if (!handleNetworkError(e))
                             break;
 
@@ -486,14 +532,14 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
             if (rmtAddrs.isEmpty() && sndError)
                 U.quietAndWarn(log, "Failed to send multicast message (is multicast enabled on this node?).");
 
-            return rmtAddrs;
+            return new T2<>(rmtAddrs, sndErr);
         }
         catch (IgniteInterruptedCheckedException ignored) {
             U.warn(log, "Got interrupted while sending address request.");
 
             Thread.currentThread().interrupt();
 
-            return rmtAddrs;
+            return new T2<>(rmtAddrs, sndErr);
         }
     }
 
@@ -610,7 +656,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
-            addrs = requestAddresses(mcastAddr, sockAddr);
+            addrs = requestAddresses(mcastAddr, sockAddr).get1();
         }
 
         /**


[09/40] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-gg-10453' into ignite-sprint-7

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-gg-10453' into ignite-sprint-7


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

Branch: refs/heads/ignite-648-failover
Commit: 15f3612d31a1f39046d4bf5cc7be6e0aed7d02c8
Parents: f45f96d d51db29
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 29 15:25:53 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 29 15:25:53 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++++--
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ------------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 6 files changed, 245 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


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


[20/40] incubator-ignite git commit: sprint-7 - Merge branch ignite-1061 into ignite-sprint-7

Posted by sb...@apache.org.
sprint-7 - Merge branch ignite-1061 into ignite-sprint-7


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

Branch: refs/heads/ignite-648-failover
Commit: 8c55f8abe8c8e56b5f28c26482027190d5e908d7
Parents: 68c21ac b467822
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 29 20:34:29 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 29 20:34:29 2015 -0700

----------------------------------------------------------------------
 .../core/src/test/config/spark/spark-config.xml | 46 ++++++++++++++++++
 .../org/apache/ignite/spark/IgniteContext.scala | 50 ++++++++++++++++++--
 .../org/apache/ignite/spark/IgniteRddSpec.scala | 18 +++++++
 3 files changed, 110 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[24/40] incubator-ignite git commit: # ignite-gg-1064 use loopback address in clock server if failed to get local host

Posted by sb...@apache.org.
# ignite-gg-1064 use loopback address in clock server if failed to get local host


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

Branch: refs/heads/ignite-648-failover
Commit: 0ef74a1449d503ae65a200e48da735b545e923da
Parents: b5bc06e
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 30 15:07:18 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 30 15:07:18 2015 +0300

----------------------------------------------------------------------
 .../processors/clock/GridClockServer.java       | 21 +++++++++++++++-----
 1 file changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ef74a14/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
index e47d1fa..a835da8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockServer.java
@@ -62,9 +62,20 @@ public class GridClockServer {
             int startPort = ctx.config().getTimeServerPortBase();
             int endPort = startPort + ctx.config().getTimeServerPortRange() - 1;
 
-            InetAddress locHost = !F.isEmpty(ctx.config().getLocalHost()) ?
-                InetAddress.getByName(ctx.config().getLocalHost()) :
-                U.getLocalHost();
+            InetAddress locHost;
+
+            if (F.isEmpty(ctx.config().getLocalHost())) {
+                try {
+                    locHost = U.getLocalHost();
+                }
+                catch (IOException e) {
+                    locHost = InetAddress.getLoopbackAddress();
+
+                    U.warn(log, "Failed to get local host address, will use loopback address: " + locHost);
+                }
+            }
+            else
+                locHost = InetAddress.getByName(ctx.config().getLocalHost());
 
             for (int p = startPort; p <= endPort; p++) {
                 try {
@@ -83,8 +94,8 @@ public class GridClockServer {
             }
 
             if (sock == null)
-                throw new IgniteCheckedException("Failed to bind time server socket within specified port range [locHost=" +
-                    locHost + ", startPort=" + startPort + ", endPort=" + endPort + ']');
+                throw new IgniteCheckedException("Failed to bind time server socket within specified port range " +
+                    "[locHost=" + locHost + ", startPort=" + startPort + ", endPort=" + endPort + ']');
         }
         catch (IOException e) {
             throw new IgniteCheckedException("Failed to start time server (failed to get local host address)", e);


[08/40] incubator-ignite git commit: encoding fix

Posted by sb...@apache.org.
encoding fix


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

Branch: refs/heads/ignite-648-failover
Commit: f45f96d0cb6e4a71c75d5020b59e9d128e02acea
Parents: 1a21477
Author: avinogradov <av...@gridgain.com>
Authored: Mon Jun 29 14:18:23 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Jun 29 14:18:23 2015 +0300

----------------------------------------------------------------------
 pom.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f45f96d0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 44cb0ce..b935b2e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -604,12 +604,12 @@
                                         </copy>
 
                                         <!-- appending filename to md5 and sha1 files. to be improved. -->
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true"> ${project.artifactId}-fabric-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true"> ${project.artifactId}-fabric-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true"> ${project.artifactId}-hadoop-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true"> ${project.artifactId}-hadoop-${project.version}-bin.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true"> ${project.artifactId}-${project.version}-src.zip</concat>
-                                        <concat destfile="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true"> ${project.artifactId}-${project.version}-src.zip</concat>
+                                        <concat destfile="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true"> ${project.artifactId}-fabric-${project.version}-bin.zip</concat>
+                                        <concat destfile="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true"> ${project.artifactId}-fabric-${project.version}-bin.zip</concat>
+                                        <concat destfile="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true"> ${project.artifactId}-hadoop-${project.version}-bin.zip</concat>
+                                        <concat destfile="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true"> ${project.artifactId}-hadoop-${project.version}-bin.zip</concat>
+                                        <concat destfile="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true"> ${project.artifactId}-${project.version}-src.zip</concat>
+                                        <concat destfile="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true"> ${project.artifactId}-${project.version}-src.zip</concat>
 
                                         <copy file="${basedir}/KEYS" todir="${basedir}/target/site" failonerror="false" />
                                     </target>


[13/40] incubator-ignite git commit: # ignite-648: fix tests (based on primaryKeysForCache)

Posted by sb...@apache.org.
# ignite-648: fix tests (based on primaryKeysForCache)


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

Branch: refs/heads/ignite-648-failover
Commit: 1146e41136c2ed76dc948b38b8437d0898fa2985
Parents: cc48ec5
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 29 20:18:31 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 29 20:18:31 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 27 +++++++++++++++++---
 1 file changed, 24 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1146e411/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 2986e02..483fc20 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -4009,10 +4009,31 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @param cnt Keys count.
      * @return Collection of keys for which given cache is primary.
      */
-    protected List<String> primaryKeysForCache(IgniteCache<String, Integer> cache, int cnt, int startFrom) {
-        List<String> found = new ArrayList<>(cnt);
+    protected List<String> primaryKeysForCache(final IgniteCache<String, Integer> cache, final int cnt, final int startFrom) {
+        Ignite grid = cache.unwrap(Ignite.class);
+
+        if (!(cache instanceof IgniteCacheProcessProxy))
+            return primaryKeysForCache0(grid.name(), cnt, startFrom);
+        else {
+            final IgniteProcessProxy proxy = (IgniteProcessProxy)grid;
+
+            final String name = proxy.name();
+
+            return proxy.remoteCompute().call(new IgniteCallable<List<String>>() {
+                @Override public List<String> call() throws Exception {
+                    return primaryKeysForCache0(name, cnt, startFrom);
+                }
+            });
+        }
+    }
+
+    private List<String> primaryKeysForCache0(String name, int cnt, int startFrom) {
+        List<String> found = new ArrayList<>();
+
+        IgniteEx ignite = grid(name);
+
+        final IgniteCache<String, Integer> cache = ignite.cache(null);
 
-        Ignite ignite = cache.unwrap(Ignite.class);
         Affinity<Object> affinity = ignite.affinity(cache.getName());
 
         for (int i = startFrom; i < startFrom + 100_000; i++) {


[37/40] incubator-ignite git commit: Merge branch 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-7


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

Branch: refs/heads/ignite-648-failover
Commit: da6ff647913a6e97a567e8108a2fefcd12709e94
Parents: cc0936f 5269bab
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 1 16:59:39 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 1 16:59:39 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 +-
 .../managers/communication/GridIoManager.java   | 49 +++++++++-------
 .../processors/cache/IgniteCacheFutureImpl.java | 42 +++++++++++++
 .../processors/cache/IgniteCacheProxy.java      |  2 +-
 .../internal/util/future/IgniteFutureImpl.java  | 18 ++++--
 .../util/nio/GridNioMessageTracker.java         | 23 +++++++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  2 +-
 .../core/src/main/resources/ignite.properties   |  2 +-
 modules/docker/Dockerfile                       | 55 +++++++++++++++++
 modules/docker/README.txt                       | 11 ++++
 modules/docker/build_users_libs.sh              | 39 ++++++++++++
 modules/docker/download_ignite.sh               | 49 ++++++++++++++++
 modules/docker/execute.sh                       | 62 ++++++++++++++++++++
 modules/docker/run.sh                           | 34 +++++++++++
 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/mesos/pom.xml                           |  2 +-
 modules/rest-http/pom.xml                       |  2 +-
 modules/scalar-2.10/pom.xml                     |  2 +-
 modules/scalar/pom.xml                          |  2 +-
 modules/schedule/pom.xml                        |  2 +-
 modules/schema-import/pom.xml                   |  2 +-
 modules/slf4j/pom.xml                           |  2 +-
 modules/spark-2.10/pom.xml                      |  2 +-
 modules/spark/pom.xml                           |  2 +-
 modules/spring/pom.xml                          |  2 +-
 modules/ssh/pom.xml                             |  2 +-
 modules/tools/pom.xml                           |  2 +-
 modules/urideploy/pom.xml                       |  2 +-
 modules/visor-console-2.10/pom.xml              |  2 +-
 modules/visor-console/pom.xml                   |  2 +-
 modules/visor-plugins/pom.xml                   |  2 +-
 modules/web/pom.xml                             |  2 +-
 modules/yardstick/pom.xml                       |  2 +-
 pom.xml                                         | 14 ++---
 49 files changed, 399 insertions(+), 73 deletions(-)
----------------------------------------------------------------------



[12/40] incubator-ignite git commit: #ignite-18: Property segmentationResolveAttempts seems to be ignored.

Posted by sb...@apache.org.
#ignite-18: Property segmentationResolveAttempts seems to be ignored.


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

Branch: refs/heads/ignite-648-failover
Commit: 1d5cfbcc6ef79dd8da99dc46a2797952dbcf4eab
Parents: 5dff89c
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Jun 29 17:50:49 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Jun 29 17:50:49 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/discovery/GridDiscoveryManager.java      | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1d5cfbcc/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 1e4b972..5e7600f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -326,6 +326,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 U.warn(log, "Configuration parameter 'segmentCheckFrequency' is too low " +
                     "(at least 2000 ms recommended): " + segChkFreq);
 
+            int segResAttemp = ctx.config().getSegmentationResolveAttempts();
+
+            if (segResAttemp < 1)
+                throw new IgniteCheckedException(
+                    "Segment resolve attempts cannot be negative or zero: " + segResAttemp);
+
             checkSegmentOnStart();
         }
 


[06/40] incubator-ignite git commit: IGNITE-1046 Added license.

Posted by sb...@apache.org.
IGNITE-1046 Added license.


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

Branch: refs/heads/ignite-648-failover
Commit: 4eef6d04c09a730188dbde33b9063965947a1683
Parents: 86c5d20
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri Jun 26 18:00:37 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Jun 26 18:00:37 2015 +0300

----------------------------------------------------------------------
 modules/docker/run.sh | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4eef6d04/modules/docker/run.sh
----------------------------------------------------------------------
diff --git a/modules/docker/run.sh b/modules/docker/run.sh
index e7feb8c..0f01c1c 100644
--- a/modules/docker/run.sh
+++ b/modules/docker/run.sh
@@ -16,13 +16,15 @@
 # limitations under the License.
 #
 
-IGNITE_VERSION=""
-
 if [ -z $SKIP_BUILD_LIBS ]; then
   ./build_users_libs.sh
 
-  IGNITE_VERSION=$(mvn -f user-repo/pom.xml dependency:list | grep ':ignite-core:jar:.*:' | \
+  PROJ_VER=$(mvn -f user-repo/pom.xml dependency:list | grep ':ignite-core:jar:.*:' | \
     sed -rn 's/.*([0-9]+\.[0-9]+\.[0-9]+).*/\1/p')
+
+  if [ ! -z $PROJ_VER ]; then
+    IGNITE_VERSION=$PROJ_VER
+  fi
 fi
 
 if [ -z $SKIP_DOWNLOAD ]; then


[16/40] incubator-ignite git commit: # ignite-648: fix near tests

Posted by sb...@apache.org.
# ignite-648: fix near tests


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

Branch: refs/heads/ignite-648-failover
Commit: 3c7ff71abdcce93f227a704c9b69f9dcb7d61d21
Parents: 1146e41
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 29 21:56:25 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 29 21:56:25 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 41 +++++++++++++++-----
 .../cache/GridCacheAbstractSelfTest.java        |  2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java | 39 ++++++++++++++-----
 .../testframework/junits/GridAbstractTest.java  |  2 +-
 .../junits/common/GridCommonAbstractTest.java   | 22 +++++++++--
 .../junits/multijvm/IgniteProcessProxy.java     | 18 +++++++++
 6 files changed, 99 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c7ff71a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 483fc20..d295be6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -2317,25 +2317,46 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             for (int g = 0; g < gridCount(); g++) {
                 for (int i = 0; i < cnt; i++) {
-                    String key = String.valueOf(i);
-
-                    GridCacheContext<String, Integer> cctx = context(g);
+                    if (!isMultiJvmAndNodeIsRemote(g))
+                        checkDeletedEntriesFlag(g, i);
+                    else {
+                        IgniteProcessProxy proxy = (IgniteProcessProxy)grid(g);
 
-                    GridCacheEntryEx entry = cctx.isNear() ? cctx.near().dht().peekEx(key) :
-                        cctx.cache().peekEx(key);
+                        final int idx = g;
+                        final int key = i;
 
-                    if (grid(0).affinity(null).mapKeyToPrimaryAndBackups(key).contains(grid(g).localNode())) {
-                        assertNotNull(entry);
-                        assertTrue(entry.deleted());
+                        proxy.remoteCompute().run(new GridAbsClosureX() {
+                            @Override public void applyx() throws IgniteCheckedException {
+                                checkDeletedEntriesFlag(idx, key);
+                            }
+                        });
                     }
-                    else
-                        assertNull(entry);
                 }
             }
         }
     }
 
     /**
+     * @param idx Grid index.
+     * @param keyN Key.
+     */
+    private void checkDeletedEntriesFlag(int idx, int keyN) {
+        String key = String.valueOf(keyN);
+
+        GridCacheContext<String, Integer> cctx = context(idx);
+
+        GridCacheEntryEx entry = cctx.isNear() ? cctx.near().dht().peekEx(key) :
+            cctx.cache().peekEx(key);
+
+        if (grid(idx).affinity(null).mapKeyToPrimaryAndBackups(key).contains(grid(idx).localNode())) {
+            assertNotNull(entry);
+            assertTrue(entry.deleted());
+        }
+        else
+            assertNull(entry);
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testRemoveLoad() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c7ff71a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index f75f77d..d98a550 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -371,7 +371,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     protected GridCacheContext<String, Integer> context(final int idx) {
         if (isMultiJvmAndNodeIsRemote(idx) && !weAreOnRemoteJvm())
             throw new UnsupportedOperationException("Operation can't be done automatically via proxy. " +
-                "Send task with this logic on remote jvm.");
+                "Send task with this logic on remote jvm instead.");
 
         return ((IgniteKernal)grid(idx)).<String, Integer>internalCache().context();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c7ff71a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 1864c0a..003c186 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.junits.multijvm.*;
 
 import java.util.*;
 
@@ -134,7 +135,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         for (int i = 0; i < gridCount(); i++)
             info(">>>>> Grid" + i + ": " + grid(i).localNode().id());
 
-        int size = 10;
+        final int size = 10;
 
         IgniteCache<Object, Object> chache0 = grid(0).cache(null);
 
@@ -147,15 +148,16 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         }
 
         for (int i = 0; i < gridCount(); i++) {
-            assertEquals(0, context(i).tm().idMapSize());
-
-            IgniteCache<Object, Object> cache = grid(i).cache(null);
-            ClusterNode node = grid(i).localNode();
-
-            for (int k = 0; k < size; k++) {
-                if (affinity(cache).isPrimaryOrBackup(node, k))
-                    assertEquals("Check failed for node: " + node.id(), k,
-                        cache.localPeek(k, CachePeekMode.ONHEAP, CachePeekMode.OFFHEAP));
+            if (!isMultiJvmAndNodeIsRemote(i))
+                putDebugCheck0(i, size);
+            else {
+                final int iCopy = i;
+
+                ((IgniteProcessProxy)grid(i)).remoteCompute().run(new IgniteRunnable() {
+                    @Override public void run() {
+                        putDebugCheck0(iCopy, size);
+                    }
+                });
             }
         }
 
@@ -172,6 +174,23 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
     }
 
     /**
+     * @param gridIdx Grid index.
+     * @param size Size.
+     */
+    private void putDebugCheck0(int gridIdx, int size) {
+        assertEquals(0, context(gridIdx).tm().idMapSize());
+
+        IgniteCache<Object, Object> cache = grid(gridIdx).cache(null);
+        ClusterNode node = grid(gridIdx).localNode();
+
+        for (int k = 0; k < size; k++) {
+            if (affinity(cache).isPrimaryOrBackup(node, k))
+                assertEquals("Check failed for node: " + node.id(), k,
+                    cache.localPeek(k, CachePeekMode.ONHEAP, CachePeekMode.OFFHEAP));
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testUnswapShort() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c7ff71a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 0a43059..fc7cc88 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -910,7 +910,7 @@ public abstract class GridAbstractTest extends TestCase {
         if (!isMultiJvm())
             return G.ignite(node.id());
         else
-            throw new UnsupportedOperationException("Operation doesn't supported yet.");
+            return IgniteProcessProxy.ignite(node.id());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c7ff71a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 9941ca8..1fc4415 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.junits.*;
+import org.apache.ignite.testframework.junits.multijvm.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
@@ -201,9 +202,24 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @param cache Cache.
      * @return {@code True} if near cache is enabled.
      */
-    protected static <K, V> boolean nearEnabled(IgniteCache<K,V> cache) {
-        CacheConfiguration cfg = ((IgniteKernal)cache.unwrap(Ignite.class)).
-            <K, V>internalCache(cache.getName()).context().config();
+    protected static <K, V> boolean nearEnabled(final IgniteCache<K,V> cache) {
+        final Ignite ignite = cache.unwrap(Ignite.class);
+
+        CacheConfiguration cfg;
+
+        if (!(ignite instanceof IgniteProcessProxy))
+            cfg = ((IgniteKernal)ignite).<K, V>internalCache(cache.getName()).context().config();
+        else {
+            final String cacheName = cache.getName();
+
+            cfg = ((IgniteProcessProxy)ignite).remoteCompute().call(new IgniteCallable<CacheConfiguration>() {
+                @Override public CacheConfiguration call() throws Exception {
+                    IgniteEx grid = IgniteNodeRunner.startedInstance();
+
+                    return ((IgniteKernal)grid).<K, V>internalCache(cacheName).context().config();
+                }
+            });
+        }
 
         return isNearEnabled(cfg);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c7ff71a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 42436e5..5ed5b87 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -131,6 +131,24 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /**
+     * @param locNodeId ID of local node the requested grid instance is managing.
+     * @return An instance of named grid. This method never returns {@code null}.
+     * @throws IgniteIllegalStateException Thrown if grid was not properly initialized or grid instance was stopped or
+     * was not started.
+     */
+    public static Ignite ignite(UUID locNodeId) {
+        A.notNull(locNodeId, "locNodeId");
+
+        for (IgniteProcessProxy ignite : gridProxies.values()) {
+            if (ignite.getId().equals(locNodeId))
+                return ignite;
+        }
+
+        throw new IgniteIllegalStateException("Grid instance with given local node ID was not properly " +
+            "started or was stopped: " + locNodeId);
+    }
+
+    /**
      * Kill all running processes.
      */
     public static void killAll() {


[19/40] incubator-ignite git commit: IGNITE-1061 - Fixed as discussed.

Posted by sb...@apache.org.
IGNITE-1061 - Fixed as discussed.


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

Branch: refs/heads/ignite-648-failover
Commit: b467822d55c8f796de2d7b3c2aa80b46f81811c1
Parents: 68c21ac
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Jun 29 20:33:20 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Jun 29 20:33:20 2015 -0700

----------------------------------------------------------------------
 .../core/src/test/config/spark/spark-config.xml | 46 ++++++++++++++++++
 .../org/apache/ignite/spark/IgniteContext.scala | 50 ++++++++++++++++++--
 .../org/apache/ignite/spark/IgniteRddSpec.scala | 18 +++++++
 3 files changed, 110 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b467822d/modules/core/src/test/config/spark/spark-config.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/spark/spark-config.xml b/modules/core/src/test/config/spark/spark-config.xml
new file mode 100644
index 0000000..4b7ffe1
--- /dev/null
+++ b/modules/core/src/test/config/spark/spark-config.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500</value>
+                                <value>127.0.0.1:47501</value>
+                                <value>127.0.0.1:47502</value>
+                                <value>127.0.0.1:47503</value>
+                                <value>127.0.0.1:47504</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b467822d/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index e52555a..5dbb1d3 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -33,11 +33,13 @@ import org.apache.spark.sql.SQLContext
  * @tparam V Value type.
  */
 class IgniteContext[K, V](
-    @scala.transient val sparkContext: SparkContext,
+    @transient val sparkContext: SparkContext,
     cfgF: () ⇒ IgniteConfiguration,
     client: Boolean = true
 ) extends Serializable with Logging {
-    @scala.transient private val driver = true
+    @transient private val driver = true
+
+    private val cfgClo = new Once(cfgF)
 
     if (!client) {
         val workers = sparkContext.getExecutorStorageStatus.length - 1
@@ -51,6 +53,15 @@ class IgniteContext[K, V](
         sparkContext.parallelize(1 to workers, workers).foreach(it ⇒ ignite())
     }
 
+    // Make sure to start Ignite on context creation.
+    ignite()
+
+    /**
+     * Creates an instance of IgniteContext with the given spring configuration.
+     *
+     * @param sc Spark context.
+     * @param springUrl Spring configuration path.
+     */
     def this(
         sc: SparkContext,
         springUrl: String
@@ -58,6 +69,17 @@ class IgniteContext[K, V](
         this(sc, () ⇒ IgnitionEx.loadConfiguration(springUrl).get1())
     }
 
+    /**
+     * Creates an instance of IgniteContext with default Ignite configuration.
+     * By default this method will use grid configuration defined in `IGNITE_HOME/config/default-config.xml`
+     * configuration file.
+     *
+     * @param sc Spark context.
+     */
+    def this(sc: SparkContext) {
+        this(sc, IgnitionEx.DFLT_CFG)
+    }
+
     val sqlContext = new SQLContext(sparkContext)
 
     /**
@@ -89,7 +111,7 @@ class IgniteContext[K, V](
      * @return Ignite instance.
      */
     def ignite(): Ignite = {
-        val igniteCfg = cfgF()
+        val igniteCfg = cfgClo()
 
         try {
             Ignition.ignite(igniteCfg.getGridName)
@@ -112,8 +134,28 @@ class IgniteContext[K, V](
      * a no-op.
      */
     def close() = {
-        val igniteCfg = cfgF()
+        val igniteCfg = cfgClo()
 
         Ignition.stop(igniteCfg.getGridName, false)
     }
 }
+
+/**
+ * Auxiliary closure that ensures that passed in closure is executed only once.
+ *
+ * @param clo Closure to wrap.
+ */
+private class Once(clo: () ⇒ IgniteConfiguration) extends Serializable {
+    @transient @volatile var res: IgniteConfiguration = null
+
+    def apply(): IgniteConfiguration = {
+        if (res == null) {
+            this.synchronized {
+                if (res == null)
+                    res = clo()
+            }
+        }
+
+        res
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b467822d/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
index 26ce693..8fa6949 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRddSpec.scala
@@ -147,6 +147,24 @@ class IgniteRddSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
                 sc.stop()
             }
         }
+
+        it("should successfully start spark context with XML configuration") {
+            val sc = new SparkContext("local[*]", "test")
+
+            try {
+                val ic = new IgniteContext[String, String](sc,
+                    "modules/core/src/test/config/spark/spark-config.xml")
+
+                val cache: IgniteRDD[String, String] = ic.fromCache(PARTITIONED_CACHE_NAME)
+
+                cache.savePairs(sc.parallelize(1 to 1000, 2).map(i ⇒ (String.valueOf(i), "val" + i)))
+
+                assert(1000 == cache.count())
+            }
+            finally {
+                sc.stop()
+            }
+        }
     }
 
     override protected def beforeEach() = {


[04/40] incubator-ignite git commit: # init marshaller cache on daemon nodes

Posted by sb...@apache.org.
# init marshaller cache on daemon nodes


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

Branch: refs/heads/ignite-648-failover
Commit: 3231f93373a2f84b7e0ea64d969b53121b6fe323
Parents: 01d842a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 26 14:34:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 26 15:28:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++++--
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 4 files changed, 245 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 65107a7..8abb135 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -899,7 +899,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** {@inheritDoc} */
     @Override public boolean clientNode() {
-        return cfg.isClientMode();
+        return cfg.isClientMode() || cfg.isDaemon();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 2f7f22c..8124594 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -265,7 +265,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         // Suppress warning if at least one ATOMIC cache found.
         perf.add("Enable ATOMIC mode if not using transactions (set 'atomicityMode' to ATOMIC)",
-                 cfg.getAtomicityMode() == ATOMIC);
+            cfg.getAtomicityMode() == ATOMIC);
 
         // Suppress warning if at least one non-FULL_SYNC mode found.
         perf.add("Disable fully synchronous writes (set 'writeSynchronizationMode' to PRIMARY_SYNC or FULL_ASYNC)",
@@ -341,8 +341,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 "Serializable transactions are disabled while default transaction isolation is SERIALIZABLE " +
                     "(most likely misconfiguration - either update 'isTxSerializableEnabled' or " +
                     "'defaultTxIsolationLevel' properties) for cache: " + U.maskName(cc.getName()),
-                    "Serializable transactions are disabled while default transaction isolation is SERIALIZABLE " +
-                        "for cache: " + U.maskName(cc.getName()));
+                "Serializable transactions are disabled while default transaction isolation is SERIALIZABLE " +
+                    "for cache: " + U.maskName(cc.getName()));
 
         if (cc.isWriteBehindEnabled()) {
             if (cfgStore == null)
@@ -527,8 +527,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings( {"unchecked"})
     @Override public void start() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
+        if (ctx.config().isDaemon()) {
+            sharedCtx = createSharedContext(ctx, CU.startStoreSessionListeners(ctx, null));
+
+            for (GridCacheSharedManager mgr : sharedCtx.managers())
+                mgr.start(sharedCtx);
+
             return;
+        }
 
         DeploymentMode depMode = ctx.config().getDeploymentMode();
 
@@ -667,8 +673,45 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
         try {
-            if (ctx.config().isDaemon())
+            if (ctx.config().isDaemon()) {
+                for (CacheConfiguration ccfg : ctx.config().getCacheConfiguration()) {
+                    if (CU.isMarshallerCache(ccfg.getName())) {
+                        CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+
+                        initialize(ccfg, cacheObjCtx);
+
+                        GridCacheContext ctx = createCache(ccfg, null, CacheType.MARSHALLER, cacheObjCtx, true);
+
+                        ctx.dynamicDeploymentId(IgniteUuid.randomUuid());
+
+                        sharedCtx.addCacheContext(ctx);
+
+                        GridCacheAdapter cache = ctx.cache();
+
+                        String name = ccfg.getName();
+
+                        caches.put(maskNull(name), cache);
+
+                        startCache(cache);
+
+                        break;
+                    }
+                }
+
+                marshallerCache().context().preloader().syncFuture().listen(new CIX1<IgniteInternalFuture<?>>() {
+                    @Override public void applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
+                        ctx.marshallerContext().onMarshallerCachePreloaded(ctx);
+                    }
+                });
+
+                for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
+                    mgr.onKernalStart();
+
+                for (GridCacheAdapter<?, ?> cache : caches.values())
+                    onKernalStart(cache);
+
                 return;
+            }
 
             ClusterNode locNode = ctx.discovery().localNode();
 
@@ -808,9 +851,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
-            return;
-
         for (String cacheName : stopSeq) {
             GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
 
@@ -850,9 +890,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @Override public void onKernalStop(boolean cancel) {
         cacheStartedLatch.countDown();
 
-        if (ctx.config().isDaemon())
-            return;
-
         for (String cacheName : stopSeq) {
             GridCacheAdapter<?, ?> cache = caches.get(maskNull(cacheName));
 
@@ -1390,6 +1427,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         Collection<DynamicCacheChangeRequest> reqs,
         AffinityTopologyVersion topVer
     ) throws IgniteCheckedException {
+        if (ctx.isDaemon())
+            return;
+
         for (DynamicCacheChangeRequest req : reqs) {
             assert req.start() : req;
             assert req.cacheType() != null : req;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
new file mode 100644
index 0000000..a0f4074
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.internal.IgniteNodeAttributes.*;
+
+/**
+ *
+ */
+public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean daemon;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDaemon(daemon);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshalOnDaemonNode1() throws Exception {
+        marshalOnDaemonNode(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshalOnDaemonNode2() throws Exception {
+        marshalOnDaemonNode(false);
+    }
+
+    /**
+     * @param startFirst If {@code true} daemon node is started first.
+     * @throws Exception If failed.
+     */
+    public void marshalOnDaemonNode(boolean startFirst) throws Exception {
+        int nodeIdx = 0;
+
+        if (!startFirst) {
+            Ignite ignite1 = startGrid(nodeIdx++);
+
+            assertFalse("true".equals(ignite1.cluster().localNode().attribute(ATTR_DAEMON)));
+        }
+
+        daemon = true;
+
+        Ignite daemonNode = startGrid(nodeIdx++);
+
+        assertEquals("true", daemonNode.cluster().localNode().attribute(ATTR_DAEMON));
+
+        daemon = false;
+
+        if (startFirst) {
+            Ignite ignite1 = startGrid(nodeIdx++);
+
+            assertFalse("true".equals(ignite1.cluster().localNode().attribute(ATTR_DAEMON)));
+        }
+
+        awaitPartitionMapExchange();
+
+        TestClass1 res1 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable1());
+
+        assertNotNull(res1);
+        assertEquals(111, res1.val);
+
+        Ignite ignite2 = startGrid(nodeIdx);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setCacheMode(REPLICATED);
+
+        IgniteCache<Object, Object> cache = ignite2.getOrCreateCache(ccfg);
+
+        awaitPartitionMapExchange();
+
+        cache.put(1, new TestClass1(1));
+        cache.put(2, new TestClass2(2));
+
+        TestClass2 res2 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable2());
+
+        assertNotNull(res2);
+        assertEquals(222, res2.val);
+    }
+
+    /**
+     *
+     */
+    private static class TestCallable1 implements IgniteCallable<TestClass1> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public TestClass1 call() throws Exception {
+            assertFalse("true".equals(ignite.cluster().localNode().attribute(ATTR_DAEMON)));
+
+            return new TestClass1(111);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCallable2 implements IgniteCallable<TestClass2> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public TestClass2 call() throws Exception {
+            assertFalse("true".equals(ignite.cluster().localNode().attribute(ATTR_DAEMON)));
+
+            return new TestClass2(222);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestClass1 implements Serializable {
+        /** */
+        public int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass1(int val) {
+            this.val = val;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestClass2 implements Serializable {
+        /** */
+        public int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass2(int val) {
+            this.val = val;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3231f933/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 6ff83e2..df22290 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
@@ -94,6 +94,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
         suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
+        suite.addTestSuite(IgniteDaemonNodeMarshallerCacheTest.class);
 
         return suite;
     }


[05/40] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-gg-10453

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-gg-10453


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

Branch: refs/heads/ignite-648-failover
Commit: d86882c3597080c62877ba36b916dda29089b900
Parents: 3231f93 6e23608
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 26 16:55:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 26 16:55:32 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 151 ++++++-----
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 103 +++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   3 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 265 ++++++++++++++++++-
 6 files changed, 451 insertions(+), 81 deletions(-)
----------------------------------------------------------------------



[39/40] incubator-ignite git commit: Merge branch 'ignite-sprint-7' into ignite-648

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-7' into ignite-648


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

Branch: refs/heads/ignite-648-failover
Commit: efb8239ee2b59f0091ae7675eeff3e8327f5ea28
Parents: ec660e5 da6ff64
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 1 17:49:09 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 1 17:49:09 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 +-
 .../ignite/compute/ComputeTaskSplitAdapter.java |   2 +-
 .../configuration/CacheConfiguration.java       | 105 +++----
 .../configuration/NearCacheConfiguration.java   |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../managers/communication/GridIoManager.java   |  49 ++--
 .../discovery/GridDiscoveryManager.java         |  32 +-
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../processors/cache/GridCacheProcessor.java    |  62 +++-
 .../processors/cache/IgniteCacheFutureImpl.java |  42 +++
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../processors/clock/GridClockServer.java       |  21 +-
 .../internal/util/future/IgniteFutureImpl.java  |  18 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../util/nio/GridNioMessageTracker.java         |  23 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  74 ++++-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/spark/spark-config.xml |  46 +++
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 +++++++++++++++++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 --------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 ++++++++++++
 .../GridP2PContinuousDeploymentSelfTest.java    |   2 -
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 modules/docker/Dockerfile                       |  55 ++++
 modules/docker/README.txt                       |  11 +
 modules/docker/build_users_libs.sh              |  39 +++
 modules/docker/download_ignite.sh               |  49 ++++
 modules/docker/execute.sh                       |  62 ++++
 modules/docker/run.sh                           |  34 +++
 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/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 .../ignite/schema/model/PojoDescriptor.java     |   2 +
 .../apache/ignite/schema/model/PojoField.java   |   1 +
 .../parser/dialect/OracleMetadataDialect.java   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |  50 +++-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 ++
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  14 +-
 72 files changed, 1225 insertions(+), 299 deletions(-)
----------------------------------------------------------------------


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


[36/40] incubator-ignite git commit: ignite-sprint-7: merge from ignite-934

Posted by sb...@apache.org.
ignite-sprint-7: merge from ignite-934


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

Branch: refs/heads/ignite-648-failover
Commit: cc0936fa76eba5556402632eef5f06325206bac0
Parents: e91bc48
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 1 16:59:14 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 1 16:59:14 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/compute/ComputeTaskSplitAdapter.java    | 2 +-
 .../org/apache/ignite/p2p/GridP2PContinuousDeploymentSelfTest.java | 2 --
 2 files changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc0936fa/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSplitAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSplitAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSplitAdapter.java
index bc4b3c2..7b3b0d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSplitAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSplitAdapter.java
@@ -73,7 +73,7 @@ public abstract class ComputeTaskSplitAdapter<T, R> extends ComputeTaskAdapter<T
 
     /** Load balancer. */
     @LoadBalancerResource
-    private transient ComputeLoadBalancer balancer;
+    private ComputeLoadBalancer balancer;
 
     /**
      * This is a simplified version of {@link ComputeTask#map(List, Object)} method.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc0936fa/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PContinuousDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PContinuousDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PContinuousDeploymentSelfTest.java
index 6b3ad63..c71ccbb 100644
--- a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PContinuousDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PContinuousDeploymentSelfTest.java
@@ -98,8 +98,6 @@ public class GridP2PContinuousDeploymentSelfTest extends GridCommonAbstractTest
      */
     @SuppressWarnings("unchecked")
     public void testDeployment() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-934");
-
         Ignite ignite = startGrid(GRID_NAME);
 
         Class cls = getExternalClassLoader().loadClass(TEST_TASK_1);