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/03 12:55:45 UTC

[15/50] [abbrv] incubator-ignite git commit: # ignite-876

# 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-gg-10299
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);