You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2016/02/05 00:43:47 UTC

[1/6] ignite git commit: Improved exception handling in IgniteContext

Repository: ignite
Updated Branches:
  refs/heads/ignite-2450 01b02ab12 -> 93dbfc498


Improved exception handling in IgniteContext


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

Branch: refs/heads/ignite-2450
Commit: d1e2957a7440b7a781439811ab9bdfedf38bd640
Parents: 500bd3a
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Feb 3 18:44:44 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Feb 3 18:44:44 2016 -0800

----------------------------------------------------------------------
 .../scala/org/apache/ignite/spark/IgniteContext.scala    | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d1e2957a/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 6e48017..57fe84f 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
@@ -20,7 +20,7 @@ package org.apache.ignite.spark
 
 import org.apache.ignite.internal.IgnitionEx
 import org.apache.ignite.internal.util.IgniteUtils
-import org.apache.ignite.{IgniteSystemProperties, Ignition, Ignite}
+import org.apache.ignite._
 import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration}
 import org.apache.spark.{Logging, SparkContext}
 import org.apache.spark.sql.SQLContext
@@ -146,14 +146,19 @@ class IgniteContext[K, V](
             Ignition.ignite(igniteCfg.getGridName)
         }
         catch {
-            case e: Exception ⇒
+            case e: IgniteIllegalStateException ⇒
                 try {
                     igniteCfg.setClientMode(client || driver)
 
                     Ignition.start(igniteCfg)
                 }
                 catch {
-                    case e: Exception ⇒ Ignition.ignite(igniteCfg.getGridName)
+                    case e: IgniteException ⇒ {
+                        logError("Failed to start Ignite client. Will try to use an existing instance with name: "
+                            + igniteCfg.getGridName, e)
+
+                        Ignition.ignite(igniteCfg.getGridName)
+                    }
                 }
         }
     }


[5/6] ignite git commit: Added cache GET benchmark.

Posted by vk...@apache.org.
Added cache GET benchmark.


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

Branch: refs/heads/ignite-2450
Commit: e2be94e62aecddeb7bf3409450a5a95a314ba186
Parents: afd3bc1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Feb 4 15:31:18 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 4 15:31:18 2016 +0300

----------------------------------------------------------------------
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |   3 +
 .../benchmarks/jmh/cache/JmhCacheBenchmark.java | 145 +++++++++++++++++++
 .../jmh/cache/JmhCachePutBenchmark.java         | 124 ----------------
 .../jmh/runner/JmhIdeBenchmarkRunner.java       |  20 ++-
 4 files changed, 160 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e2be94e6/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheAbstractBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheAbstractBenchmark.java
index e8829bb..709ab77 100644
--- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheAbstractBenchmark.java
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheAbstractBenchmark.java
@@ -58,6 +58,9 @@ public class JmhCacheAbstractBenchmark extends JmhAbstractBenchmark {
     /** Default amount of nodes. */
     protected static final int DFLT_DATA_NODES = 1;
 
+    /** Items count. */
+    protected static final int CNT = 100000;
+
     /** IP finder shared across nodes. */
     private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2be94e6/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheBenchmark.java
new file mode 100644
index 0000000..f55d16c
--- /dev/null
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCacheBenchmark.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.benchmarks.jmh.cache;
+
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.internal.benchmarks.jmh.runner.JmhIdeBenchmarkRunner;
+import org.apache.ignite.internal.benchmarks.model.IntValue;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.profile.GCProfiler;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Put benchmark.
+ */
+@SuppressWarnings("unchecked")
+public class JmhCacheBenchmark extends JmhCacheAbstractBenchmark {
+    /**
+     * Set up routine.
+     *
+     * @throws Exception If failed.
+     */
+
+    public void setup() throws Exception {
+        super.setup();
+
+        IgniteDataStreamer<Integer, IntValue> dataLdr = node.dataStreamer(cache.getName());
+
+        for (int i = 0; i < CNT; i++)
+            dataLdr.addData(i, new IntValue(i));
+
+        dataLdr.close();
+
+        System.out.println("Cache populated.");
+    }
+
+    /**
+     * Test PUT operation.
+     *
+     * @throws Exception If failed.
+     */
+    @Benchmark
+    public void put() throws Exception {
+        int key = ThreadLocalRandom.current().nextInt(CNT);
+
+        cache.put(key, new IntValue(key));
+    }
+
+    /**
+     * Test PUT operation.
+     *
+     * @throws Exception If failed.
+     */
+    @Benchmark
+    public Object get() throws Exception {
+        int key = ThreadLocalRandom.current().nextInt(CNT);
+
+        return cache.get(key);
+    }
+
+    /**
+     * Run benchmarks.
+     *
+     * @param args Arguments.
+     * @throws Exception If failed.
+     */
+    public static void main(String[] args) throws Exception {
+        run("put", CacheAtomicityMode.ATOMIC);
+        run("get", CacheAtomicityMode.ATOMIC);
+        run("put", CacheAtomicityMode.TRANSACTIONAL);
+        run("get", CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * Run benchmarks for atomic cache.
+     *
+     * @param benchmark Benchmark name.
+     * @param atomicityMode Atomicity mode.
+     * @throws Exception If failed.
+     */
+    private static void run(String benchmark, CacheAtomicityMode atomicityMode) throws Exception {
+        run(benchmark, 4, true, atomicityMode, CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        run(benchmark, 4, true, atomicityMode, CacheWriteSynchronizationMode.FULL_SYNC);
+        run(benchmark, 4, false, atomicityMode, CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        run(benchmark, 4, false, atomicityMode, CacheWriteSynchronizationMode.FULL_SYNC);
+    }
+
+    /**
+     * Run benchmark.
+     *
+     * @param benchmark Benchmark to run.
+     * @param threads Amount of threads.
+     * @param client Client mode flag.
+     * @param atomicityMode Atomicity mode.
+     * @param writeSyncMode Write synchronization mode.
+     * @throws Exception If failed.
+     */
+    private static void run(String benchmark, int threads, boolean client, CacheAtomicityMode atomicityMode,
+        CacheWriteSynchronizationMode writeSyncMode) throws Exception {
+        String simpleClsName = JmhCacheBenchmark.class.getSimpleName();
+
+        String output = simpleClsName + "-" + benchmark +
+            "-" + threads + "-threads" +
+            "-" + (client ? "client" : "data") +
+            "-" + atomicityMode +
+            "-" + writeSyncMode;
+
+        JmhIdeBenchmarkRunner.create()
+            .forks(1)
+            .threads(threads)
+            .warmupIterations(10)
+            .measurementIterations(60)
+            .benchmarks(simpleClsName + "." + benchmark)
+            .output(output + ".jmh.log")
+            .profilers(GCProfiler.class)
+            .jvmArguments(
+                "-Xms4g",
+                "-Xmx4g",
+                "-XX:+UnlockCommercialFeatures",
+                "-XX:+FlightRecorder",
+                "-XX:StartFlightRecording=delay=30s,dumponexit=true,settings=alloc,filename=" + output + ".jfr",
+                JmhIdeBenchmarkRunner.createProperty(PROP_ATOMICITY_MODE, atomicityMode),
+                JmhIdeBenchmarkRunner.createProperty(PROP_WRITE_SYNC_MODE, writeSyncMode),
+                JmhIdeBenchmarkRunner.createProperty(PROP_DATA_NODES, 2),
+                JmhIdeBenchmarkRunner.createProperty(PROP_CLIENT_MODE, client))
+            .run();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2be94e6/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCachePutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCachePutBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCachePutBenchmark.java
deleted file mode 100644
index 848e7ce..0000000
--- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/cache/JmhCachePutBenchmark.java
+++ /dev/null
@@ -1,124 +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.benchmarks.jmh.cache;
-
-import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.internal.benchmarks.jmh.runner.JmhIdeBenchmarkRunner;
-import org.apache.ignite.internal.benchmarks.model.IntValue;
-import org.openjdk.jmh.annotations.Benchmark;
-import org.openjdk.jmh.profile.GCProfiler;
-
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * Put benchmark.
- */
-@SuppressWarnings("unchecked")
-public class JmhCachePutBenchmark extends JmhCacheAbstractBenchmark {
-    /** Items count. */
-    private static final int CNT = 100000;
-
-    /**
-     * Set up routine.
-     *
-     * @throws Exception If failed.
-     */
-
-    public void setup() throws Exception {
-        super.setup();
-
-        IgniteDataStreamer<Integer, IntValue> dataLdr = node.dataStreamer(cache.getName());
-
-        for (int i = 0; i < CNT; i++)
-            dataLdr.addData(i, new IntValue(i));
-
-        dataLdr.close();
-
-        System.out.println("Cache populated.");
-    }
-
-    /**
-     * Test PUT operation.
-     *
-     * @throws Exception If failed.
-     */
-    @Benchmark
-    public void testPut() throws Exception {
-        int key = ThreadLocalRandom.current().nextInt(CNT);
-
-        cache.put(key, new IntValue(key));
-    }
-
-    /**
-     * Run benchmarks.
-     *
-     * @param args Arguments.
-     * @throws Exception If failed.
-     */
-    public static void main(String[] args) throws Exception {
-        run(CacheAtomicityMode.ATOMIC);
-    }
-
-    /**
-     * Run benchmarks for atomic cache.
-     *
-     * @param atomicityMode Atomicity mode.
-     * @throws Exception If failed.
-     */
-    private static void run(CacheAtomicityMode atomicityMode) throws Exception {
-        run(4, true, atomicityMode, CacheWriteSynchronizationMode.PRIMARY_SYNC);
-        run(4, true, atomicityMode, CacheWriteSynchronizationMode.FULL_SYNC);
-        run(4, false, atomicityMode, CacheWriteSynchronizationMode.PRIMARY_SYNC);
-        run(4, false, atomicityMode, CacheWriteSynchronizationMode.FULL_SYNC);
-    }
-
-    /**
-     * Run benchmark.
-     *
-     * @param client Client mode flag.
-     * @param writeSyncMode Write synchronization mode.
-     * @throws Exception If failed.
-     */
-    private static void run(int threads, boolean client, CacheAtomicityMode atomicityMode,
-        CacheWriteSynchronizationMode writeSyncMode) throws Exception {
-        String output = "ignite-cache-put-" + threads + "-threads-" + (client ? "client" : "data") +
-            "-" + atomicityMode + "-" + writeSyncMode;
-
-        JmhIdeBenchmarkRunner.create()
-            .forks(1)
-            .threads(threads)
-            .warmupIterations(10)
-            .measurementIterations(60)
-            .classes(JmhCachePutBenchmark.class)
-            .output(output + ".jmh.log")
-            .profilers(GCProfiler.class)
-            .jvmArguments(
-                "-Xms4g",
-                "-Xmx4g",
-                "-XX:+UnlockCommercialFeatures",
-                "-XX:+FlightRecorder",
-                "-XX:StartFlightRecording=delay=30s,dumponexit=true,settings=alloc,filename=" + output + ".jfr",
-                JmhIdeBenchmarkRunner.createProperty(PROP_ATOMICITY_MODE, atomicityMode),
-                JmhIdeBenchmarkRunner.createProperty(PROP_WRITE_SYNC_MODE, writeSyncMode),
-                JmhIdeBenchmarkRunner.createProperty(PROP_DATA_NODES, 2),
-                JmhIdeBenchmarkRunner.createProperty(PROP_CLIENT_MODE, client))
-            .run();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e2be94e6/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/runner/JmhIdeBenchmarkRunner.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/runner/JmhIdeBenchmarkRunner.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/runner/JmhIdeBenchmarkRunner.java
index af84862..0cad088 100644
--- a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/runner/JmhIdeBenchmarkRunner.java
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/runner/JmhIdeBenchmarkRunner.java
@@ -42,8 +42,8 @@ public class JmhIdeBenchmarkRunner {
     /** Output time unit. */
     private TimeUnit outputTimeUnit = TimeUnit.SECONDS;
 
-    /** Classes to run. */
-    private Class[] clss;
+    /** Benchmarks to run. */
+    private Object[] benchmarks;
 
     /** JVM arguments. */
     private String[] jvmArgs;
@@ -123,11 +123,11 @@ public class JmhIdeBenchmarkRunner {
     }
 
     /**
-     * @param clss Classes.
+     * @param benchmarks Benchmarks.
      * @return This instance.
      */
-    public JmhIdeBenchmarkRunner classes(Class... clss) {
-        this.clss = clss;
+    public JmhIdeBenchmarkRunner benchmarks(Object... benchmarks) {
+        this.benchmarks = benchmarks;
 
         return this;
     }
@@ -191,9 +191,13 @@ public class JmhIdeBenchmarkRunner {
                 builder.getBenchModes().add(benchmarkMode);
         }
 
-        if (clss != null) {
-            for (Class cls : clss)
-                builder.include(cls.getSimpleName());
+        if (benchmarks != null) {
+            for (Object benchmark : benchmarks) {
+                if (benchmark instanceof Class)
+                    builder.include(((Class)benchmark).getSimpleName());
+                else
+                    builder.include(benchmark.toString());
+            }
         }
 
         if (jvmArgs != null)


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

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


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

Branch: refs/heads/ignite-2450
Commit: 93dbfc498fb4fc1240bf9897286f021e8c142891
Parents: 01b02ab e2be94e
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Feb 4 15:23:17 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Feb 4 15:23:17 2016 -0800

----------------------------------------------------------------------
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |   3 +
 .../benchmarks/jmh/cache/JmhCacheBenchmark.java | 145 +++++++++++++++++++
 .../jmh/cache/JmhCachePutBenchmark.java         | 124 ----------------
 .../jmh/runner/JmhIdeBenchmarkRunner.java       |  20 ++-
 .../processors/cache/GridCacheMapEntry.java     |   4 +-
 .../cache/GridCacheUpdateAtomicResult.java      |   4 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java |  28 +---
 .../org/apache/ignite/spark/IgniteContext.scala |  11 +-
 .../ignite/internal/GridFactorySelfTest.java    |   2 +-
 10 files changed, 176 insertions(+), 167 deletions(-)
----------------------------------------------------------------------



[3/6] ignite git commit: IGNITE-2541: Fixed potential NPE in GridCacheUpdateAtomicResult caused by unsafe [long -> Long -> long] transitions.

Posted by vk...@apache.org.
IGNITE-2541: Fixed potential NPE in GridCacheUpdateAtomicResult caused by unsafe [long -> Long -> long] transitions.


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

Branch: refs/heads/ignite-2450
Commit: 10a2b7a18de3d9e0cb3fdf147956c127fa4d4d2b
Parents: 74d9d05
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Feb 4 09:41:26 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 4 09:41:26 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheMapEntry.java      | 4 ++--
 .../internal/processors/cache/GridCacheUpdateAtomicResult.java   | 4 ++--
 .../cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java  | 2 +-
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/10a2b7a1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 64cfd01..ae40295 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -2135,7 +2135,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         null,
                         null,
                         false,
-                        updateCntr0 == null ? 0 : updateCntr);
+                        updateCntr0 == null ? 0 : updateCntr0);
                 }
             }
             else
@@ -2431,7 +2431,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             enqueueVer,
             conflictCtx,
             true,
-            updateCntr0);
+            updateCntr0 == null ? 0 : updateCntr0);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a2b7a1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
index 9df476e..2355b7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
@@ -58,7 +58,7 @@ public class GridCacheUpdateAtomicResult {
     private final boolean sndToDht;
 
     /** */
-    private final Long updateCntr;
+    private final long updateCntr;
 
     /** Value computed by entry processor. */
     private IgniteBiTuple<Object, Exception> res;
@@ -137,7 +137,7 @@ public class GridCacheUpdateAtomicResult {
     /**
      * @return Partition update index.
      */
-    public Long updateCounter() {
+    public long updateCounter() {
         return updateCntr;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10a2b7a1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index e31af19..06c8441 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -225,7 +225,7 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
         @Nullable GridCacheVersion conflictVer,
         boolean addPrevVal,
         @Nullable CacheObject prevVal,
-        @Nullable Long updateCntr) {
+        long updateCntr) {
         AffinityTopologyVersion topVer = updateReq.topologyVersion();
 
         Collection<ClusterNode> dhtNodes = cctx.dht().topology().nodes(entry.partition(), topVer);


[2/6] ignite git commit: Fixed GridFactorySelfTest which started failing due to minor change in exception message.

Posted by vk...@apache.org.
Fixed GridFactorySelfTest which started failing due to minor change in exception message.


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

Branch: refs/heads/ignite-2450
Commit: 74d9d05b7cee071dad4c50a3a053781d78a488a8
Parents: d1e2957
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Feb 4 09:36:52 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 4 09:36:52 2016 +0300

----------------------------------------------------------------------
 .../test/java/org/apache/ignite/internal/GridFactorySelfTest.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74d9d05b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
index d72b577..cca3e8b 100644
--- a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java
@@ -469,7 +469,7 @@ public class GridFactorySelfTest extends GridCommonAbstractTest {
                                 String msg = e.getMessage();
 
                                 if (msg != null &&
-                                    (msg.contains("Default grid instance has already been started.") ||
+                                    (msg.contains("Default Ignite instance has already been started.") ||
                                     msg.contains("Ignite instance with this name has already been started:")))
                                     info("Caught expected exception: " + msg);
                                 else


[4/6] ignite git commit: IGNITE-2540: Removed unnecessary ArrayList from GridNioServer.DirectNioClientWorker write processing methods.

Posted by vk...@apache.org.
IGNITE-2540: Removed unnecessary ArrayList from GridNioServer.DirectNioClientWorker write processing methods.


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

Branch: refs/heads/ignite-2450
Commit: afd3bc1e3c249ae86071f0d15b41c6eedbdeb050
Parents: 10a2b7a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Feb 4 11:13:36 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 4 11:13:36 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioServer.java | 28 ++------------------
 1 file changed, 2 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/afd3bc1e/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 84c8157..c7679c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -980,8 +980,6 @@ public class GridNioServer<T> {
 
                 NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
 
-                List<NioOperationFuture<?>> doneFuts = null;
-
                 while (true) {
                     if (req == null) {
                         req = (NioOperationFuture<?>)ses.pollFuture();
@@ -1012,10 +1010,7 @@ public class GridNioServer<T> {
 
                     // Fill up as many messages as possible to write buffer.
                     while (finished) {
-                        if (doneFuts == null)
-                            doneFuts = new ArrayList<>();
-
-                        doneFuts.add(req);
+                        req.onDone();
 
                         req = (NioOperationFuture<?>)ses.pollFuture();
 
@@ -1059,13 +1054,6 @@ public class GridNioServer<T> {
                     if (!skipWrite) {
                         int cnt = sockCh.write(buf);
 
-                        if (!F.isEmpty(doneFuts)) {
-                            for (int i = 0; i < doneFuts.size(); i++)
-                                doneFuts.get(i).onDone();
-
-                            doneFuts.clear();
-                        }
-
                         if (log.isTraceEnabled())
                             log.trace("Bytes sent [sockCh=" + sockCh + ", cnt=" + cnt + ']');
 
@@ -1185,13 +1173,8 @@ public class GridNioServer<T> {
             }
 
             // Fill up as many messages as possible to write buffer.
-            List<NioOperationFuture<?>> doneFuts = null;
-
             while (finished) {
-                if (doneFuts == null)
-                    doneFuts = new ArrayList<>();
-
-                doneFuts.add(req);
+                req.onDone();
 
                 req = (NioOperationFuture<?>)ses.pollFuture();
 
@@ -1218,13 +1201,6 @@ public class GridNioServer<T> {
             if (!skipWrite) {
                 int cnt = sockCh.write(buf);
 
-                if (!F.isEmpty(doneFuts)) {
-                    for (int i = 0; i < doneFuts.size(); i++)
-                        doneFuts.get(i).onDone();
-
-                    doneFuts.clear();
-                }
-
                 if (log.isTraceEnabled())
                     log.trace("Bytes sent [sockCh=" + sockCh + ", cnt=" + cnt + ']');