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/08 08:20:50 UTC

[1/8] incubator-ignite git commit: ignite-1070 Zombie entries in cache with SortedEvictionPolicy

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-747 ccce9fb23 -> c15d964a5


ignite-1070 Zombie entries in cache with SortedEvictionPolicy


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

Branch: refs/heads/ignite-747
Commit: 9d0aa6f1244e13187b64dec81b8d33d94f3db2af
Parents: 44506c3
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 7 14:49:13 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 7 14:49:13 2015 +0300

----------------------------------------------------------------------
 .../cache/eviction/fifo/FifoEvictionPolicy.java  |  5 -----
 .../cache/eviction/lru/LruEvictionPolicy.java    |  5 -----
 .../eviction/sorted/SortedEvictionPolicy.java    | 19 +++++--------------
 ...cheConcurrentEvictionConsistencySelfTest.java | 15 +++++++++++++--
 4 files changed, 18 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d0aa6f1/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
index 221bc39..1e06ffd 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
@@ -216,11 +216,6 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
                 // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(node))
                     return false;
-                else {
-                    memSize.add(-entry.size());
-
-                    return true;
-                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d0aa6f1/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
index 0be26c8..ba6f88a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
@@ -200,11 +200,6 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
                 // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(node))
                     return false;
-                else {
-                    memSize.add(-entry.size());
-
-                    return true;
-                }
             }
         }
         else if (queue.unlinkx(node)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d0aa6f1/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
index b8b82fa..d99b3b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
@@ -224,21 +224,17 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     private boolean touch(EvictableEntry<K, V> entry) {
         Holder<K, V> holder = entry.meta();
 
-        // Entry has not been add yet to backed queue..
+        // Entry has not been added yet to backed queue.
         if (holder == null) {
             while (true) {
                 holder = new Holder<>(entry, orderCnt.incrementAndGet());
 
-                set.add(holder);
+                if (entry.putMetaIfAbsent(holder) != null)
+                    return false; // Set has not been changed.
 
-                if (entry.putMetaIfAbsent(holder) != null) {
-                    // Was concurrently added, need to remove it from queue.
-                    removeHolder(holder);
+                set.add(holder);
 
-                    // Set has not been changed.
-                    return false;
-                }
-                else if (holder.order > 0) {
+                if (holder.order > 0) {
                     if (!entry.isCached()) {
                         // Was concurrently evicted, need to remove it from queue.
                         removeHolder(holder);
@@ -253,11 +249,6 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
                 // If holder was removed by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(holder))
                     return false;
-                else {
-                    memSize.add(-entry.size());
-
-                    return true;
-                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d0aa6f1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
index 921422e..91e9ecc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.cache.eviction.sorted.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -273,15 +274,25 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
             info("Test results [threadCnt=" + threadCnt + ", iterCnt=" + ITERATION_CNT + ", cacheSize=" + cache.size() +
                 ", internalQueueSize" + queue.size() + ", duration=" + (System.currentTimeMillis() - start) + ']');
 
+            boolean detached = false;
+
             for (Cache.Entry<Integer, Integer> e : queue) {
                 Integer rmv = cache.getAndRemove(e.getKey());
 
-                if (rmv == null)
-                    fail("Eviction policy contains key that is not present in cache: " + e);
+                CacheEvictableEntryImpl unwrapped = e.unwrap(CacheEvictableEntryImpl.class);
+
+                if (rmv == null && (unwrapped.meta() != null || unwrapped.isCached())) {
+                    U.warn(log, "Detached entry: " + e);
+
+                    detached = true;
+                }
                 else
                     info("Entry removed: " + rmv);
             }
 
+            if (detached)
+                fail("Eviction policy contains keys that are not present in cache");
+
             if (!(cache.localSize() == 0)) {
                 boolean zombies = false;
 


[7/8] incubator-ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-747
Commit: 0f1b31a5b89f7406bfe44e488a9a832567ee1ccb
Parents: 17ee8a6 4f46ee8
Author: ashutak <as...@gridgain.com>
Authored: Tue Jul 7 20:39:42 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jul 7 20:39:42 2015 +0300

----------------------------------------------------------------------
 .../internal/interop/InteropIgnition.java       |  17 +-
 .../internal/interop/InteropProcessor.java      |   7 +
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 13 files changed, 384 insertions(+), 73 deletions(-)
----------------------------------------------------------------------



[4/8] incubator-ignite git commit: ignite-1031 Query metrics updated incorrectly

Posted by sb...@apache.org.
ignite-1031 Query metrics updated incorrectly


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

Branch: refs/heads/ignite-747
Commit: c04cba9e82d3f9a7a5d851047de92322c54cc568
Parents: 9d0aa6f
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 7 19:28:38 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 7 19:28:38 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 11 files changed, 361 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 5b82c34..c2425f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -36,6 +36,7 @@ import org.jetbrains.annotations.*;
 
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.*;
 
 /**
@@ -413,12 +414,19 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     @SuppressWarnings("IfMayBeConditional")
     private <R> CacheQueryFuture<R> execute(@Nullable IgniteReducer<T, R> rmtReducer,
         @Nullable IgniteClosure<T, R> rmtTransform, @Nullable Object... args) {
-        Collection<ClusterNode> nodes = nodes();
+        Collection<ClusterNode> nodes;
+
+        try {
+            nodes = nodes();
+        }
+        catch (IgniteCheckedException e) {
+            return queryErrorFuture(cctx, e, log);
+        }
 
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (nodes.isEmpty())
-            return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), new ClusterGroupEmptyCheckedException());
+            return queryErrorFuture(cctx, new ClusterGroupEmptyCheckedException(), log);
 
         if (log.isDebugEnabled())
             log.debug("Executing query [query=" + this + ", nodes=" + nodes + ']');
@@ -429,7 +437,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                 cctx.deploy().registerClasses(args);
             }
             catch (IgniteCheckedException e) {
-                return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
+                return queryErrorFuture(cctx, e, log);
             }
         }
 
@@ -457,7 +465,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     /**
      * @return Nodes to execute on.
      */
-    private Collection<ClusterNode> nodes() {
+    private Collection<ClusterNode> nodes() throws IgniteCheckedException {
         CacheMode cacheMode = cctx.config().getCacheMode();
 
         switch (cacheMode) {
@@ -466,6 +474,10 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                     U.warn(log, "Ignoring query projection because it's executed over LOCAL cache " +
                         "(only local node will be queried): " + this);
 
+                if (type == SCAN && cctx.config().getCacheMode() == LOCAL &&
+                    partition() != null && partition() >= cctx.affinity().partitions())
+                    throw new IgniteCheckedException("Invalid partition number: " + partition());
+
                 return Collections.singletonList(cctx.localNode());
 
             case REPLICATED:
@@ -513,6 +525,21 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         });
     }
 
+    /**
+     * @param cctx Cache context.
+     * @param e Exception.
+     * @param log Logger.
+     */
+    private static <T> GridCacheQueryErrorFuture<T> queryErrorFuture(GridCacheContext<?, ?> cctx,
+        Exception e, IgniteLogger log) {
+
+        GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
+
+        GridQueryProcessor.onExecuted(cctx, metrics, null, e, 0, 0, log);
+
+        return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheQueryAdapter.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e080c6d..6af0f40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1453,6 +1453,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             throw (IgniteCheckedException)err;
         }
+        catch (Exception e) {
+            err = e;
+
+            throw new IgniteCheckedException(e);
+        }
         finally {
             GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
index 63912bf..3eb96b6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
@@ -33,7 +33,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
  */
 public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstractTest {
     /** Grid count. */
-    private static final int GRID_CNT = 2;
+    protected int gridCnt;
 
     /** Cache mode. */
     protected CacheMode cacheMode;
@@ -43,7 +43,7 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
+        startGridsMultiThreaded(gridCnt);
     }
 
     /** {@inheritDoc} */
@@ -122,6 +122,57 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
+     * Test metrics for failed SQL queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlFieldsQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from UNKNOWN");
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
      * Test metrics for Scan queries.
      *
      * @throws Exception In case of error.
@@ -163,6 +214,57 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
+     * Test metrics for failed Scan queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testScanQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        ScanQuery<String, Integer> qry = new ScanQuery<>(Integer.MAX_VALUE);
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
      * Test metrics for SQL cross cache queries.
      *
      * @throws Exception In case of error.
@@ -202,4 +304,55 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
         assertTrue(m.maximumTime() >= 0);
         assertTrue(m.minimumTime() >= 0);
     }
+
+    /**
+     * Test metrics for failed SQL cross cache queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlCrossCacheQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"G\".Integer");
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
new file mode 100644
index 0000000..d28d3a0
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for local cache query metrics.
+ */
+public class CacheLocalQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = LOCAL;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java
new file mode 100644
index 0000000..f54a091
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for partitioned distributed cache query metrics.
+ */
+public class CachePartitionedQueryMetricsDistributedSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 2;
+        cacheMode = PARTITIONED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java
new file mode 100644
index 0000000..efeb5d3
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for partitioned local cache query metrics.
+ */
+public class CachePartitionedQueryMetricsLocalSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = PARTITIONED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
deleted file mode 100644
index 666acfb..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,32 +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 static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests for partitioned cache query metrics.
- */
-public class CachePartitionedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        cacheMode = PARTITIONED;
-
-        super.beforeTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java
new file mode 100644
index 0000000..ae2a420
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for replicated distributed cache query metrics.
+ */
+public class CacheReplicatedQueryMetricsDistributedSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 2;
+        cacheMode = REPLICATED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java
new file mode 100644
index 0000000..d411f9c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for replicated local cache query metrics.
+ */
+public class CacheReplicatedQueryMetricsLocalSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = REPLICATED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
deleted file mode 100644
index 2be6dd3..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,32 +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 static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests for replicated cache query metrics.
- */
-public class CacheReplicatedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        cacheMode = REPLICATED;
-
-        super.beforeTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 181ff0c..2d7d0ce 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -116,8 +116,11 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(H2CompareBigQueryTest.class);
 
         // Cache query metrics.
-        suite.addTestSuite(CachePartitionedQueryMetricsSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsSelfTest.class);
+        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
 
         //Unmarshallig query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);


[2/8] incubator-ignite git commit: GG-10469: Done.

Posted by sb...@apache.org.
GG-10469: Done.


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

Branch: refs/heads/ignite-747
Commit: 96fa3002904fcad50dafc1f19655c2d44eaece40
Parents: 96c9695
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Jul 7 18:07:05 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Jul 7 18:07:05 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/interop/InteropIgnition.java   | 17 ++++++++++++++++-
 .../ignite/internal/interop/InteropProcessor.java  |  7 +++++++
 2 files changed, 23 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96fa3002/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java
index 2989a17..faae72b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java
@@ -50,6 +50,9 @@ public class InteropIgnition {
      */
     public static synchronized InteropProcessor start(@Nullable String springCfgPath, @Nullable String gridName,
         int factoryId, long envPtr, long dataPtr) {
+        if (envPtr <= 0)
+            throw new IgniteException("Environment pointer must be positive.");
+
         ClassLoader oldClsLdr = Thread.currentThread().getContextClassLoader();
 
         Thread.currentThread().setContextClassLoader(InteropIgnition.class.getClassLoader());
@@ -83,13 +86,25 @@ public class InteropIgnition {
      * Get instance by environment pointer.
      *
      * @param gridName Grid name.
-     * @return Instance or {@code null} if it doesn't exists (never started or stopped).
+     * @return Instance or {@code null} if it doesn't exist (never started or stopped).
      */
     @Nullable public static synchronized InteropProcessor instance(@Nullable String gridName) {
         return instances.get(gridName);
     }
 
     /**
+     * Get environment pointer of the given instance.
+     *
+     * @param gridName Grid name.
+     * @return Environment pointer or {@code 0} in case grid with such name doesn't exist.
+     */
+    public static synchronized long environmentPointer(@Nullable String gridName) {
+        InteropProcessor proc = instance(gridName);
+
+        return proc != null ? proc.environmentPointer() : 0;
+    }
+
+    /**
      * Stop single instance.
      *
      * @param gridName Grid name,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96fa3002/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropProcessor.java
index 325a464..94b733f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropProcessor.java
@@ -32,6 +32,13 @@ public interface InteropProcessor {
     public Ignite ignite();
 
     /**
+     * Get environment pointer associated with this processor.
+     *
+     * @return Environment pointer.
+     */
+    public long environmentPointer();
+
+    /**
      * Get stop runnable to perform cleanup when interop is not longer used.
      * <p/>
      * <b>NOTE!</b> This runnable is called when current instance of interop processor is eligible for garbage


[5/8] incubator-ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite

Posted by sb...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-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/4f46ee88
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4f46ee88
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4f46ee88

Branch: refs/heads/ignite-747
Commit: 4f46ee886d4b1a40c71d0cbdab273b9eb38e3717
Parents: c04cba9 6ed5c1c
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 7 19:29:18 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 7 19:29:18 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/interop/InteropIgnition.java   | 17 ++++++++++++++++-
 .../ignite/internal/interop/InteropProcessor.java  |  7 +++++++
 2 files changed, 23 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[8/8] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-747

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


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

Branch: refs/heads/ignite-747
Commit: c15d964a5aabdd969f784f3daeeb20346415c465
Parents: ccce9fb 0f1b31a
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 8 09:19:47 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 8 09:19:47 2015 +0300

----------------------------------------------------------------------
 dev-tools/slurp.sh                              |   2 +-
 .../cache/eviction/fifo/FifoEvictionPolicy.java |   5 -
 .../cache/eviction/lru/LruEvictionPolicy.java   |   5 -
 .../eviction/sorted/SortedEvictionPolicy.java   |  19 +--
 .../internal/interop/InteropIgnition.java       |  17 +-
 .../internal/interop/InteropProcessor.java      |   7 +
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 ...heConcurrentEvictionConsistencySelfTest.java |  15 +-
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 scripts/git-patch-prop.sh                       |   2 +-
 19 files changed, 404 insertions(+), 101 deletions(-)
----------------------------------------------------------------------



[3/8] incubator-ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-747
Commit: 6ed5c1c694dbdf40840aaf7457f883959e8aa917
Parents: 96fa300 9d0aa6f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Jul 7 18:07:55 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Jul 7 18:07:55 2015 +0300

----------------------------------------------------------------------
 .../cache/eviction/fifo/FifoEvictionPolicy.java |   5 -
 .../cache/eviction/lru/LruEvictionPolicy.java   |   5 -
 .../eviction/sorted/SortedEvictionPolicy.java   |  19 +--
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  26 +++
 .../cache/CacheFutureExceptionSelfTest.java     | 171 +++++++++++++++++++
 .../GridCachePartitionedNodeRestartTest.java    |   5 -
 ...ePartitionedOptimisticTxNodeRestartTest.java |   2 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 -
 .../GridCacheReplicatedNodeRestartSelfTest.java |   5 -
 ...heConcurrentEvictionConsistencySelfTest.java |  15 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 11 files changed, 217 insertions(+), 42 deletions(-)
----------------------------------------------------------------------



[6/8] incubator-ignite git commit: # master: update DEFAULT_BRANCH to master.

Posted by sb...@apache.org.
# master: update DEFAULT_BRANCH to master.


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

Branch: refs/heads/ignite-747
Commit: 17ee8a6b837c6266b5065b956df0918305ec8814
Parents: 9d0aa6f
Author: ashutak <as...@gridgain.com>
Authored: Tue Jul 7 20:39:14 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jul 7 20:39:14 2015 +0300

----------------------------------------------------------------------
 dev-tools/slurp.sh        | 2 +-
 scripts/git-patch-prop.sh | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ee8a6b/dev-tools/slurp.sh
----------------------------------------------------------------------
diff --git a/dev-tools/slurp.sh b/dev-tools/slurp.sh
index 7edc776..ad7bc01 100755
--- a/dev-tools/slurp.sh
+++ b/dev-tools/slurp.sh
@@ -24,7 +24,7 @@
 #
 # Default branch name. Need to use last version of "jiraslurp" scripts.
 #
-DEFAULT_BRANCH='ignite-sprint-5'
+DEFAULT_BRANCH='master'
 
 #
 # TC URL. It is 10.30.0.229 for public TC from agents.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ee8a6b/scripts/git-patch-prop.sh
----------------------------------------------------------------------
diff --git a/scripts/git-patch-prop.sh b/scripts/git-patch-prop.sh
index 16f907e..1b531fa 100644
--- a/scripts/git-patch-prop.sh
+++ b/scripts/git-patch-prop.sh
@@ -19,6 +19,6 @@
 #
 # Git patch-file maker/applier properties.
 #
-IGNITE_DEFAULT_BRANCH='ignite-sprint-7'
+IGNITE_DEFAULT_BRANCH='master'
 
 PATCHES_HOME=${IGNITE_HOME}