Alexey Kukushkin created IGNITE-16137:
-----------------------------------------

             Summary: ContainsKey operation fetches entry value into heap
                 Key: IGNITE-16137
                 URL: https://issues.apache.org/jira/browse/IGNITE-16137
             Project: Ignite
          Issue Type: Improvement
          Components: cache
    Affects Versions: 2.11
            Reporter: Alexey Kukushkin


[See similar problem for the remove() 
operation|https://issues.apache.org/jira/browse/IGNITE-15959]

{{IgniteCache#containsKey(key)}} operation fetches full entry into heap memory. 
This is inefficient when working with large objects: our application running 
with limited heap memory fails with {{java.lang.OutOfMemoryError: Java heap 
space}} when trying to check if a key exists.

It seems wrong that Ignite needs to fetch the full entry on heap to check if 
the key exists. Please enhance Ignite to not be doing that or explain why 
Ignite must do that.
h2. Reproducer
h3. Steps

Create a Gradle project with the below class and run it as
{{./gradlew test --tests apache.ignite.issues.ContainsOperationHeapUsage}}

{{build.gradle}}
{code:groovy}
test {
    minHeapSize = "512m"
    maxHeapSize = "512m"
}
{code}
{{ContainsOperationHeapUsage.java}}
{code:java}
public class ContainsOperationHeapUsage {
    /** Run with -Xmx512m -Xms512m */
    @Test
    public void containsOperationFetchesValueOnHeap() {
        var igniteCfg = new IgniteConfiguration()
            .setDiscoverySpi(
                new TcpDiscoverySpi()
                    .setIpFinder(new 
TcpDiscoveryVmIpFinder().setAddresses(Collections.singleton("127.0.0.1:47500")))
            )
            .setCacheConfiguration(new CacheConfiguration<>("blobs"));

        try (var ignite = Ignition.start(igniteCfg)) {
            Cache<Integer, byte[]> cache = ignite.cache("blobs");

            // Put a BLOB having size of 35% of free memory to the cache
            Runtime.getRuntime().gc();
            var freeMemory = Runtime.getRuntime().freeMemory();
            var blobSize = (int)(freeMemory * 0.35);
            putBlob(cache, blobSize);

            // Use 70% of the free heap
            Runtime.getRuntime().gc();
            var unused = new byte[2 * blobSize];

            // Check if the blob exists in the cache.
            // This throws "OutOfMemoryError: Java heap space" since Ignite 
retrieves full entry to the heap.
            // Why does Ignite retrieve entry value to check if the key exists?
            cache.containsKey(1);
        }
    }

    private static void putBlob(Cache<Integer, byte[]> cache, int blobSize) {
        var blob = new byte[blobSize];
        cache.put(1, blob);
    }
}
{code}

h3. Expected

The test passes

h3. Actual

The {{cache.containsKey}} operatoin fails with:
{noformat}
java.lang.OutOfMemoryError: Java heap space
        at 
org.apache.ignite.internal.processors.cache.IncompleteCacheObject.<init>(IncompleteCacheObject.java:44)
        at 
org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.toCacheObject(CacheObjectBinaryProcessorImpl.java:1385)
        at 
org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.readIncompleteValue(CacheDataRowAdapter.java:680)
        at 
org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.readFragment(CacheDataRowAdapter.java:500)
        at 
org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.readIncomplete(CacheDataRowAdapter.java:411)
        at 
org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.doInitFromLink(CacheDataRowAdapter.java:316)
        at 
org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.initFromLink(CacheDataRowAdapter.java:165)
        at 
org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.initFromLink(CacheDataRowAdapter.java:136)
        at 
org.apache.ignite.internal.processors.cache.tree.DataRow.<init>(DataRow.java:55)
        at 
org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore.dataRow(CacheDataRowStore.java:129)
        at 
org.apache.ignite.internal.processors.cache.tree.CacheDataTree.getRow(CacheDataTree.java:422)
        at 
org.apache.ignite.internal.processors.cache.tree.CacheDataTree.getRow(CacheDataTree.java:63)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$GetOne.found(BPlusTree.java:3156)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Search.run0(BPlusTree.java:317)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$GetPageHandler.run(BPlusTree.java:5921)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Search.run(BPlusTree.java:290)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$GetPageHandler.run(BPlusTree.java:5907)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler.readPage(PageHandler.java:174)
        at 
org.apache.ignite.internal.processors.cache.persistence.DataStructure.read(DataStructure.java:397)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.read(BPlusTree.java:6108)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.findDown(BPlusTree.java:1446)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.doFind(BPlusTree.java:1413)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.findOne(BPlusTree.java:1379)
        at 
org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.findOne(BPlusTree.java:1364)
        at 
org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl.find(IgniteCacheOffheapManagerImpl.java:2815)
        at 
org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl.read(IgniteCacheOffheapManagerImpl.java:633)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture.localGet(GridPartitionedSingleGetFuture.java:483)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture.tryLocalGet(GridPartitionedSingleGetFuture.java:445)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture.mapKeyToNode(GridPartitionedSingleGetFuture.java:409)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture.map(GridPartitionedSingleGetFuture.java:284)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture.init(GridPartitionedSingleGetFuture.java:248)
        at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.getAsync0(GridDhtAtomicCache.java:1456)
{noformat}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to