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/06/02 20:09:52 UTC

[1/7] incubator-ignite git commit: # ignite-876

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-5 2b052c29c -> 1bd6da1a1


# ignite-876


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

Branch: refs/heads/ignite-sprint-5
Commit: 68d5bd8a6ae31af1870b2e8d83eb070e8774e8e3
Parents: 5c30f9c
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 1 12:51:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 1 17:28:45 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  62 +++++-
 .../cache/distributed/dht/GridDhtGetFuture.java |  11 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  27 +++
 .../GridCachePartitionedFullApiSelfTest.java    |  32 +++
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   2 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    | 200 +++++++++++++++++++
 .../cache/GridCacheOffheapIndexGetSelfTest.java |  18 +-
 .../IgniteCacheWithIndexingTestSuite.java       |   1 +
 8 files changed, 337 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/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 92035af..0743aaa 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
@@ -61,6 +61,9 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     private static final byte IS_UNSWAPPED_MASK = 0x02;
 
     /** */
+    private static final byte IS_OFFHEAP_PTR_MASK = 0x04;
+
+    /** */
     public static final GridCacheAtomicVersionComparator ATOMIC_VER_COMPARATOR = new GridCacheAtomicVersionComparator();
 
     /**
@@ -433,6 +436,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                         if (e.offheapPointer() > 0) {
                             offHeapPointer(e.offheapPointer());
 
+                            flags |= IS_OFFHEAP_PTR_MASK;
+
                             if (needVal) {
                                 CacheObject val = cctx.fromOffheap(offHeapPointer(), false);
 
@@ -498,7 +503,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 return;
             }
 
-            if (val == null && cctx.offheapTiered() && hasOffHeapPointer()) {
+            if (cctx.offheapTiered() && hasOffHeapPointer()) {
                 if (log.isDebugEnabled())
                     log.debug("Value did not change, skip write swap entry: " + this);
 
@@ -509,10 +514,16 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             }
 
             IgniteUuid valClsLdrId = null;
+            IgniteUuid keyClsLdrId = null;
 
-            if (val != null) {
-                valClsLdrId = cctx.deploy().getClassLoaderId(
-                    val.value(cctx.cacheObjectContext(), false).getClass().getClassLoader());
+            if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) {
+                if (val != null) {
+                    valClsLdrId = cctx.deploy().getClassLoaderId(
+                        U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false)));
+                }
+
+                keyClsLdrId = cctx.deploy().getClassLoaderId(
+                    U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
             }
 
             IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();
@@ -523,7 +534,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 ver,
                 ttlExtras(),
                 expireTime,
-                cctx.deploy().getClassLoaderId(U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))),
+                keyClsLdrId,
                 valClsLdrId);
 
             if (log.isDebugEnabled())
@@ -3617,6 +3628,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                         return true;
                     }
+                    else
+                        evictFailed(prev);
                 }
             }
             else {
@@ -3660,8 +3673,11 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                             return true;
                         }
-                        else
+                        else {
+                            evictFailed(prevVal);
+
                             return false;
+                        }
                     }
                 }
             }
@@ -3680,6 +3696,25 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         return false;
     }
 
+    /**
+     * @param prevVal Previous value.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void evictFailed(CacheObject prevVal) throws IgniteCheckedException {
+        if (cctx.offheapTiered() && ((flags & IS_OFFHEAP_PTR_MASK) != 0)) {
+            cctx.swap().removeOffheap(key());
+
+            offHeapPointer(0);
+
+            flags &= ~IS_OFFHEAP_PTR_MASK;
+
+            GridCacheQueryManager qryMgr = cctx.queries();
+
+            if (qryMgr != null)
+                qryMgr.onUnswap(key, prevVal);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer)
         throws IgniteCheckedException {
@@ -3692,10 +3727,17 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
                 if (!isStartVersion() && hasValueUnlocked()) {
                     IgniteUuid valClsLdrId = null;
+                    IgniteUuid keyClsLdrId = null;
 
-                    if (val != null)
-                        valClsLdrId = cctx.deploy().getClassLoaderId(
-                            U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false)));
+                    if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) {
+                        if (val != null) {
+                            valClsLdrId = cctx.deploy().getClassLoaderId(
+                                U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false)));
+                        }
+
+                        keyClsLdrId = cctx.deploy().getClassLoaderId(
+                            U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
+                    }
 
                     IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();
 
@@ -3706,7 +3748,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                         ver,
                         ttlExtras(),
                         expireTimeExtras(),
-                        cctx.deploy().getClassLoaderId(U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))),
+                        keyClsLdrId,
                         valClsLdrId);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index f6f930e..742fbfe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -295,6 +295,11 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     if (info == null)
                         continue;
 
+                    boolean addReader = (!e.deleted() && k.getValue() && !skipVals);
+
+                    if (addReader)
+                        e.unswap(false);
+
                     // Register reader. If there are active transactions for this entry,
                     // then will wait for their completion before proceeding.
                     // TODO: GG-4003:
@@ -303,8 +308,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     // TODO: To fix, check that reader is contained in the list of readers once
                     // TODO: again after the returned future completes - if not, try again.
                     // TODO: Also, why is info read before transactions are complete, and not after?
-                    IgniteInternalFuture<Boolean> f = (!e.deleted() && k.getValue() && !skipVals) ?
-                        e.addReader(reader, msgId, topVer) : null;
+                    IgniteInternalFuture<Boolean> f = addReader ? e.addReader(reader, msgId, topVer) : null;
 
                     if (f != null) {
                         if (txFut == null)
@@ -317,6 +321,9 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
 
                     break;
                 }
+                catch (IgniteCheckedException err) {
+                    return new GridFinishedFuture<>(err);
+                }
                 catch (GridCacheEntryRemovedException ignore) {
                     if (log.isDebugEnabled())
                         log.debug("Got removed entry when getting a DHT value: " + e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/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 70d8f9c..25f31ae 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
@@ -3913,6 +3913,33 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testIterator() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(null);
+
+        final int KEYS = 1000;
+
+        for (int i = 0; i < KEYS; i++)
+            cache.put(i, i);
+
+        // Try to initialize readers in case when near cache is enabled.
+        for (int i = 0; i < gridCount(); i++) {
+            cache = grid(i).cache(null);
+
+            for (int k = 0; k < KEYS; k++)
+                assertEquals((Object)k, cache.get(k));
+        }
+
+        int cnt = 0;
+
+        for (Cache.Entry e : cache)
+            cnt++;
+
+        assertEquals(KEYS, cnt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testIgniteCacheIterator() throws Exception {
         IgniteCache<String, Integer> cache = jcache(0);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
index 7e16ac7..da5d1bb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 
+import javax.cache.*;
+
 import static org.apache.ignite.cache.CacheMode.*;
 
 /**
@@ -72,4 +75,33 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp
         for (int i = 0 ; i < aff.partitions(); i++)
             String.valueOf(cache.entrySet(i));
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUpdate() throws Exception {
+        if (gridCount() > 1) {
+            IgniteCache<Object, Object> cache = grid(0).cache(null);
+
+            Integer key = nearKey(cache);
+
+            primaryCache(key, null).put(key, 1);
+
+            assertEquals(1, cache.get(key));
+
+            primaryCache(key, null).put(key, 2);
+
+            if (cache.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null)
+                assertEquals(2, cache.localPeek(key));
+
+            assertEquals(2, cache.get(key));
+
+            int cnt = 0;
+
+            for (Cache.Entry e : cache)
+                cnt++;
+
+            assertEquals(1, cnt);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
index 4392365..61c3563 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
@@ -65,7 +65,7 @@ public class GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest extends G
         cache.put(key, 5); // Put from near to add reader on primary.
 
         assertEquals(5, primaryCache.localPeek(key, CachePeekMode.ONHEAP).intValue());
-        assertEquals(5, primaryCache.localPeek(key, CachePeekMode.OFFHEAP).intValue());
+        assertNull(primaryCache.localPeek(key, CachePeekMode.OFFHEAP));
         assertEquals(5, cache.get(key).intValue());
         assertEquals(5, map.get(key));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java
new file mode 100644
index 0000000..8afd746
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class GridCacheOffheapIndexEntryEvictTest extends GridCommonAbstractTest {
+    /** */
+    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setNetworkTimeout(2000);
+
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setBackups(1);
+        cacheCfg.setOffHeapMaxMemory(0);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setMemoryMode(OFFHEAP_TIERED);
+        cacheCfg.setEvictionPolicy(null);
+        cacheCfg.setSqlOnheapRowCacheSize(10);
+        cacheCfg.setIndexedTypes(Integer.class, TestValue.class);
+        cacheCfg.setNearConfiguration(null);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryWhenLocked() throws Exception {
+        IgniteCache<Integer, TestValue> cache = grid(0).cache(null);
+
+        List<Lock> locks = new ArrayList<>();
+
+        final int ENTRIES = 1000;
+
+        try {
+            for (int i = 0; i < ENTRIES; i++) {
+                cache.put(i, new TestValue(i));
+
+                Lock lock = cache.lock(i);
+
+                lock.lock(); // Lock entry so that it should not be evicted.
+
+                locks.add(lock);
+
+                for (int j = 0; j < 3; j++)
+                    assertNotNull(cache.get(i));
+            }
+
+            checkQuery(cache, "_key >= 100", ENTRIES - 100);
+        }
+        finally {
+            for (Lock lock : locks)
+                lock.unlock();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUpdates() throws Exception {
+        final int ENTRIES = 500;
+
+        IgniteCache<Integer, TestValue> cache = grid(0).cache(null);
+
+        for (int i = 0; i < ENTRIES; i++) {
+            for (int j = 0; j < 3; j++) {
+                cache.getAndPut(i, new TestValue(i));
+
+                assertNotNull(cache.get(i));
+
+                assertNotNull(cache.localPeek(i));
+            }
+
+            checkQuery(cache, "_key >= 0", i + 1);
+        }
+
+        for (int i = 0; i < ENTRIES; i++) {
+            if (i % 2 == 0)
+                cache.getAndRemove(i);
+            else
+                cache.remove(i);
+
+            checkQuery(cache, "_key >= 0", ENTRIES - (i + 1));
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param sql Query.
+     * @param expCnt Number of expected entries.
+     */
+    private void checkQuery(IgniteCache<Integer, TestValue> cache, String sql, int expCnt) {
+        SqlQuery<Integer, TestValue> qry = new SqlQuery<>(TestValue.class, sql);
+
+        List<Cache.Entry<Integer, TestValue>> res = cache.query(qry).getAll();
+
+        assertEquals(expCnt, res.size());
+
+        for (Cache.Entry<Integer, TestValue> e : res) {
+            assertNotNull(e.getKey());
+
+            assertEquals((int)e.getKey(), e.getValue().val);
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestValue implements Externalizable {
+        /** */
+        private int val;
+
+        /**
+         *
+         */
+        public TestValue() {
+            // No-op.
+        }
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            val = in.readInt();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
index 4e40040..41eb45a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -25,6 +26,9 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import javax.cache.*;
+import java.util.*;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheMemoryMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
@@ -67,7 +71,6 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
         cacheCfg.setMemoryMode(OFFHEAP_TIERED);
         cacheCfg.setEvictionPolicy(null);
-        cacheCfg.setOffHeapMaxMemory(OFFHEAP_MEM);
         cacheCfg.setIndexedTypes(Long.class, Long.class);
 
         cfg.setCacheConfiguration(cacheCfg);
@@ -98,8 +101,6 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testGet() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-873");
-
         IgniteCache<Long, Long> cache = grid(0).cache(null);
 
         for (long i = 0; i < 100; i++)
@@ -107,5 +108,16 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
 
         for (long i = 0; i < 100; i++)
             assertEquals((Long)i, cache.get(i));
+
+        SqlQuery<Long, Long> qry = new SqlQuery<>(Long.class, "_val >= 90");
+
+        List<Cache.Entry<Long, Long>> res = cache.query(qry).getAll();
+
+        assertEquals(10, res.size());
+
+        for (Cache.Entry<Long, Long> e : res) {
+            assertNotNull(e.getKey());
+            assertNotNull(e.getValue());
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68d5bd8a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index ae45120..240caff 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -47,6 +47,7 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite {
         suite.addTestSuite(CacheTtlOnheapAtomicPartitionedSelfTest.class);
 
         suite.addTestSuite(GridCacheOffheapIndexGetSelfTest.class);
+        suite.addTestSuite(GridCacheOffheapIndexEntryEvictTest.class);
 
         suite.addTestSuite(CacheConfigurationP2PTest.class);
 


[2/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-876-2

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


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

Branch: refs/heads/ignite-sprint-5
Commit: 7f7e9613daada938046e0c63cdb382075472486b
Parents: 68d5bd8 97d0b04
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 1 17:30:42 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 1 17:30:42 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |   10 +-
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   20 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   17 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |    5 +
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |    2 -
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 scripts/git-format-patch.sh                     |   14 +-
 223 files changed, 17549 insertions(+), 10246 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f7e9613/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------


[7/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-876-2' into ignite-sprint-5

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


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

Branch: refs/heads/ignite-sprint-5
Commit: 1bd6da1a1a9e2b2e5408f9ff24ca1e1a280a91ca
Parents: 2b052c2 c74ddc4
Author: sboikov <se...@inria.fr>
Authored: Tue Jun 2 21:04:15 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Tue Jun 2 21:04:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  64 +++++-
 .../cache/distributed/dht/GridDhtGetFuture.java |  11 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  27 +++
 .../GridCachePartitionedFullApiSelfTest.java    |  32 +++
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   2 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    | 200 +++++++++++++++++++
 .../cache/GridCacheOffheapIndexGetSelfTest.java |  18 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |  37 ++++
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java |  37 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +-
 .../IgniteCacheWithIndexingTestSuite.java       |   1 +
 11 files changed, 377 insertions(+), 54 deletions(-)
----------------------------------------------------------------------



[6/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-876-2

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


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

Branch: refs/heads/ignite-sprint-5
Commit: c74ddc421efafddae3eced31183f989b3803cc49
Parents: c1f0ce2 4f3788d
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 11:56:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 11:56:55 2015 +0300

----------------------------------------------------------------------
 .../hibernate/CacheHibernatePersonStore.java    | 202 +---------
 .../hibernate/CacheHibernateStoreExample.java   |  17 +
 .../store/jdbc/CacheJdbcPersonStore.java        | 180 ++-------
 .../store/jdbc/CacheJdbcStoreExample.java       |  13 +
 .../store/spring/CacheSpringPersonStore.java    | 128 ++++++
 .../store/spring/CacheSpringStoreExample.java   | 143 +++++++
 .../datagrid/store/spring/package-info.java     |  22 ++
 .../apache/ignite/cache/store/CacheStore.java   |   2 +
 .../ignite/cache/store/CacheStoreSession.java   |  22 ++
 .../cache/store/CacheStoreSessionListener.java  | 133 +++++++
 .../jdbc/CacheJdbcStoreSessionListener.java     | 141 +++++++
 .../configuration/CacheConfiguration.java       |  32 ++
 .../configuration/IgniteConfiguration.java      |  38 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../managers/communication/GridIoManager.java   |  12 +-
 .../processors/cache/GridCacheProcessor.java    |  12 +-
 .../cache/GridCacheSharedContext.java           |  47 ++-
 .../processors/cache/GridCacheUtils.java        |  54 +++
 .../cache/store/CacheOsStoreManager.java        |   1 -
 .../cache/store/CacheStoreManager.java          |   7 +-
 .../store/GridCacheStoreManagerAdapter.java     | 202 ++++++++--
 .../cache/transactions/IgniteTxAdapter.java     |  33 +-
 .../transactions/IgniteTxLocalAdapter.java      | 142 ++++---
 ...cheStoreSessionListenerAbstractSelfTest.java | 315 +++++++++++++++
 ...heStoreSessionListenerLifecycleSelfTest.java | 395 +++++++++++++++++++
 .../CacheJdbcStoreSessionListenerSelfTest.java  | 175 ++++++++
 .../IgniteCrossCacheTxStoreSelfTest.java        | 147 ++++---
 .../loadtests/hashmap/GridCacheTestContext.java |   3 +-
 .../junits/cache/TestCacheSession.java          |  18 +
 .../cache/TestThreadLocalCacheSession.java      |  15 +
 .../junits/common/GridCommonAbstractTest.java   |  24 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 .../CacheHibernateStoreSessionListener.java     | 216 ++++++++++
 ...heHibernateStoreSessionListenerSelfTest.java | 228 +++++++++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +-
 modules/spring/pom.xml                          |  14 +
 .../spring/CacheSpringStoreSessionListener.java | 207 ++++++++++
 ...CacheSpringStoreSessionListenerSelfTest.java | 197 +++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   3 +
 40 files changed, 3057 insertions(+), 492 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c74ddc42/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 12134d6,e435ed7..2d5fed5
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -59,10 -59,10 +59,10 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
          suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
 -        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.class);
 +        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+         // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
          suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
  //        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class); TODO IGNITE-484


[4/7] incubator-ignite git commit: # ignite-876

Posted by sb...@apache.org.
# ignite-876


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

Branch: refs/heads/ignite-sprint-5
Commit: 21b6a033e0f3dce5f9ea9949142eaee7d264edb4
Parents: 1dc248d
Author: sboikov <se...@inria.fr>
Authored: Mon Jun 1 21:25:26 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Mon Jun 1 21:25:26 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java         | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/21b6a033/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 6aae7e9..4680994 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
@@ -3700,18 +3700,20 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
      * @param prevVal Previous value.
      * @throws IgniteCheckedException If failed.
      */
-    private void evictFailed(CacheObject prevVal) throws IgniteCheckedException {
+    private void evictFailed(@Nullable CacheObject prevVal) throws IgniteCheckedException {
         if (cctx.offheapTiered() && ((flags & IS_OFFHEAP_PTR_MASK) != 0)) {
-            cctx.swap().removeOffheap(key());
+            flags &= ~IS_OFFHEAP_PTR_MASK;
 
-            value(prevVal);
+            if (prevVal != null) {
+                cctx.swap().removeOffheap(key());
 
-            flags &= ~IS_OFFHEAP_PTR_MASK;
+                value(prevVal);
 
-            GridCacheQueryManager qryMgr = cctx.queries();
+                GridCacheQueryManager qryMgr = cctx.queries();
 
-            if (qryMgr != null)
-                qryMgr.onUnswap(key, prevVal);
+                if (qryMgr != null)
+                    qryMgr.onUnswap(key, prevVal);
+            }
         }
     }
 


[3/7] incubator-ignite git commit: # ignite-876

Posted by sb...@apache.org.
# ignite-876


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

Branch: refs/heads/ignite-sprint-5
Commit: 1dc248d8d91ce855e9fd756fb221a082a6b347a6
Parents: 7f7e961
Author: sboikov <se...@inria.fr>
Authored: Mon Jun 1 19:21:58 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Mon Jun 1 19:21:58 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheMapEntry.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1dc248d8/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 5f60b5f..6aae7e9 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
@@ -3704,7 +3704,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (cctx.offheapTiered() && ((flags & IS_OFFHEAP_PTR_MASK) != 0)) {
             cctx.swap().removeOffheap(key());
 
-            offHeapPointer(0);
+            value(prevVal);
 
             flags &= ~IS_OFFHEAP_PTR_MASK;
 


[5/7] incubator-ignite git commit: # ignite-876 renamed test

Posted by sb...@apache.org.
# ignite-876 renamed test


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

Branch: refs/heads/ignite-sprint-5
Commit: c1f0ce2df5087051b07ce1552b8a2a85097f0fa5
Parents: 21b6a03
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 10:45:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 10:45:01 2015 +0300

----------------------------------------------------------------------
 ...QueryMultiThreadedOffHeapTieredSelfTest.java | 37 ++++++++++++++++++++
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java | 37 --------------------
 .../IgniteCacheQuerySelfTestSuite.java          |  2 +-
 3 files changed, 38 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1f0ce2d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
new file mode 100644
index 0000000..df4c01d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ * Test queries in off-heap tiered mode.
+ */
+public class IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest extends IgniteCacheQueryMultiThreadedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = super.cacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.REPLICATED);
+        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+        ccfg.setOffHeapMaxMemory(0);
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1f0ce2d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
deleted file mode 100644
index 8b09d0f..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
+++ /dev/null
@@ -1,37 +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.cache.*;
-import org.apache.ignite.configuration.*;
-
-/**
- * Test queries in off-heap tired mode.
- */
-public class IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest extends IgniteCacheQueryMultiThreadedSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration() {
-        CacheConfiguration ccfg = super.cacheConfiguration();
-
-        ccfg.setCacheMode(CacheMode.REPLICATED);
-        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
-        ccfg.setOffHeapMaxMemory(0);
-
-        return ccfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1f0ce2d/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 f42963a..12134d6 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
@@ -59,7 +59,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
         suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);