[ 
https://issues.apache.org/jira/browse/IGNITE-21396?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Aleksandr Polovtcev updated IGNITE-21396:
-----------------------------------------
    Description: 
I've created a following test:

{code:java}
public class TestClass extends ClusterPerTestIntegrationTest {
    @Test
    void testConcurrentDrop() {
        String tableName = "TEST";

        executeSql(String.format("CREATE TABLE %s (key INT PRIMARY KEY)", 
tableName));

        var stopFlag = new AtomicBoolean();

        CompletableFuture<Void> selectFuture = CompletableFuture.runAsync(() -> 
{
            while (!stopFlag.get()) {
                executeSql(String.format("SELECT COUNT(*) FROM %s", tableName));
            }
        });

        executeSql(String.format("DROP TABLE %s", tableName));

        stopFlag.set(true);

        assertThat(selectFuture, willCompleteSuccessfully());
    }
}
{code}

It starts a polling thread that uses {{SELECT COUNT(*)}} to constantly poll a 
table, while we drop the table in a different thread. This test fails with an 
NPE in SqlQueryProcessor:

{code:java}
Caused by: java.lang.NullPointerException: table
        at java.base/java.util.Objects.requireNonNull(Objects.java:235)
        at 
org.apache.ignite.internal.sql.engine.SqlQueryProcessor.primaryReplicas(SqlQueryProcessor.java:378)
        at 
org.apache.ignite.internal.sql.engine.SqlQueryProcessor$1.forTable(SqlQueryProcessor.java:313)
        at 
org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.lambda$mapFragments$4(MappingServiceImpl.java:160)
        at 
java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
        at 
java.base/java.util.AbstractList$RandomAccessSpliterator.forEachRemaining(AbstractList.java:720)
        at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at 
java.base/java.util.stream.StreamSpliterators$WrappingSpliterator.forEachRemaining(StreamSpliterators.java:310)
        at 
java.base/java.util.stream.Streams$ConcatSpliterator.forEachRemaining(Streams.java:734)
        at 
java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
        at 
java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
        at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
        at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
        at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
        at 
java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921)
        at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at 
java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:682)
        at 
org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.mapFragments(MappingServiceImpl.java:168)
        at 
org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.lambda$map0$2(MappingServiceImpl.java:140)
        at 
com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$remap$15(BoundedLocalCache.java:2629)
        at 
java.base/java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1916)
        at 
com.github.benmanes.caffeine.cache.BoundedLocalCache.remap(BoundedLocalCache.java:2624)
        at 
com.github.benmanes.caffeine.cache.BoundedLocalCache.compute(BoundedLocalCache.java:2574)
        at 
com.github.benmanes.caffeine.cache.LocalCache.compute(LocalCache.java:93)
        at 
org.apache.ignite.internal.sql.engine.util.cache.CaffeineCacheFactory$CaffeineCacheToCacheAdapter.compute(CaffeineCacheFactory.java:124)
        at 
org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.map0(MappingServiceImpl.java:126)
        at 
org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.map(MappingServiceImpl.java:101)
        at 
org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImpl$DistributedQueryManager.execute(ExecutionServiceImpl.java:827)
        at 
org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImpl.executeQuery(ExecutionServiceImpl.java:266)
        at 
org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImpl.executePlan(ExecutionServiceImpl.java:300)
        at 
org.apache.ignite.internal.sql.engine.SqlQueryProcessor.executePlan(SqlQueryProcessor.java:681)
        at 
org.apache.ignite.internal.sql.engine.SqlQueryProcessor.lambda$executeParsedStatement$17(SqlQueryProcessor.java:644)
{code}

I would expect the test to fail with a different error, e.g. that the table has 
not been found.


> NPE SqlQueryProcessor during concurrent SELECT and DROP TABLE
> -------------------------------------------------------------
>
>                 Key: IGNITE-21396
>                 URL: https://issues.apache.org/jira/browse/IGNITE-21396
>             Project: Ignite
>          Issue Type: Bug
>          Components: sql
>            Reporter: Aleksandr Polovtcev
>            Priority: Major
>              Labels: ignite-3
>
> I've created a following test:
> {code:java}
> public class TestClass extends ClusterPerTestIntegrationTest {
>     @Test
>     void testConcurrentDrop() {
>         String tableName = "TEST";
>         executeSql(String.format("CREATE TABLE %s (key INT PRIMARY KEY)", 
> tableName));
>         var stopFlag = new AtomicBoolean();
>         CompletableFuture<Void> selectFuture = CompletableFuture.runAsync(() 
> -> {
>             while (!stopFlag.get()) {
>                 executeSql(String.format("SELECT COUNT(*) FROM %s", 
> tableName));
>             }
>         });
>         executeSql(String.format("DROP TABLE %s", tableName));
>         stopFlag.set(true);
>         assertThat(selectFuture, willCompleteSuccessfully());
>     }
> }
> {code}
> It starts a polling thread that uses {{SELECT COUNT(*)}} to constantly poll a 
> table, while we drop the table in a different thread. This test fails with an 
> NPE in SqlQueryProcessor:
> {code:java}
> Caused by: java.lang.NullPointerException: table
>       at java.base/java.util.Objects.requireNonNull(Objects.java:235)
>       at 
> org.apache.ignite.internal.sql.engine.SqlQueryProcessor.primaryReplicas(SqlQueryProcessor.java:378)
>       at 
> org.apache.ignite.internal.sql.engine.SqlQueryProcessor$1.forTable(SqlQueryProcessor.java:313)
>       at 
> org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.lambda$mapFragments$4(MappingServiceImpl.java:160)
>       at 
> java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
>       at 
> java.base/java.util.AbstractList$RandomAccessSpliterator.forEachRemaining(AbstractList.java:720)
>       at 
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
>       at 
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
>       at 
> java.base/java.util.stream.StreamSpliterators$WrappingSpliterator.forEachRemaining(StreamSpliterators.java:310)
>       at 
> java.base/java.util.stream.Streams$ConcatSpliterator.forEachRemaining(Streams.java:734)
>       at 
> java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
>       at 
> java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:276)
>       at 
> java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
>       at 
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
>       at 
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
>       at 
> java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921)
>       at 
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>       at 
> java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:682)
>       at 
> org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.mapFragments(MappingServiceImpl.java:168)
>       at 
> org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.lambda$map0$2(MappingServiceImpl.java:140)
>       at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$remap$15(BoundedLocalCache.java:2629)
>       at 
> java.base/java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1916)
>       at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache.remap(BoundedLocalCache.java:2624)
>       at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache.compute(BoundedLocalCache.java:2574)
>       at 
> com.github.benmanes.caffeine.cache.LocalCache.compute(LocalCache.java:93)
>       at 
> org.apache.ignite.internal.sql.engine.util.cache.CaffeineCacheFactory$CaffeineCacheToCacheAdapter.compute(CaffeineCacheFactory.java:124)
>       at 
> org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.map0(MappingServiceImpl.java:126)
>       at 
> org.apache.ignite.internal.sql.engine.exec.mapping.MappingServiceImpl.map(MappingServiceImpl.java:101)
>       at 
> org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImpl$DistributedQueryManager.execute(ExecutionServiceImpl.java:827)
>       at 
> org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImpl.executeQuery(ExecutionServiceImpl.java:266)
>       at 
> org.apache.ignite.internal.sql.engine.exec.ExecutionServiceImpl.executePlan(ExecutionServiceImpl.java:300)
>       at 
> org.apache.ignite.internal.sql.engine.SqlQueryProcessor.executePlan(SqlQueryProcessor.java:681)
>       at 
> org.apache.ignite.internal.sql.engine.SqlQueryProcessor.lambda$executeParsedStatement$17(SqlQueryProcessor.java:644)
> {code}
> I would expect the test to fail with a different error, e.g. that the table 
> has not been found.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to