You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Vyacheslav Koptilin (JIRA)" <ji...@apache.org> on 2018/07/24 12:46:00 UTC

[jira] [Updated] (IGNITE-8689) SQL query execution may lead to NullPointerException while node is stopped

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

Vyacheslav Koptilin updated IGNITE-8689:
----------------------------------------
    Description: 
Let's consider the following scenario:
 * Start a new node (node 'A') and create a new partitioned cache that resides on that node

{code:java}
Ignite ignite = Ignition.start("examples/config/segmentation/node-A.xml");
IgniteCache<String, String> cache = ignite.getOrCreateCache(new CacheConfiguration<String, String>()
    .setName("default")
    .setIndexedTypes(String.class, String.class)
    .setNodeFilter(new NodeFilter())
);

public class NodeFilter implements IgnitePredicate<ClusterNode> {
    @Override public boolean apply(ClusterNode node) {
        return node.attribute("test.attribute").equals("first-node");
    }
}{code}
 * Start the second node (node 'B') with a custom connector configuration:

{code:java}
<property name="clientConnectorConfiguration">
    <bean class="org.apache.ignite.configuration.ClientConnectorConfiguration">
        <property name="host" value="127.0.0.1"/>
        <property name="port" value="22222"/>
        <property name="portRange" value="10"/>
    </bean>
</property>

Ignite ignite = Ignition.start("examples/config/segmentation/node-B.xml");

Executors.newScheduledThreadPool(1).schedule(
    new Runnable() {
        @Override public void run() {
            DiscoverySpi spi = ignite.configuration().getDiscoverySpi();
            spi.failNode(ignite.cluster().localNode().id(), "test message");
        }
    },
    30,
    TimeUnit.SECONDS);{code}
 * Execute simple SQL query using sqlline for example (JDBC driver should be connected to the node 'B')

{code:java}
./sqlline.sh --verbose=true -u jdbc:ignite:thin://127.0.0.1:22222

select * from UNKNOWN_TABLE;{code}
In that case, {{IgniteH2Indexing.prepareStatement()}} throws {{SQLException(Table not found)}} and the implementation (see {{IgniteH2Indexing.prepareStatementAndCaches()}}) tries to start caches that are not started yet by sending {{ClientCacheChangeDummyDiscoveryMessage}} to 'discovery-worker' thread,
which in turn posts that message to 'exchange-worker' thread.

Assume that while processing of {{ClientCacheChangeDummyDiscoveryMessage}} by the 'exchange-worker', the discovery thread receives {{EVT_NODE_FAILED}} (as a result of segmentation) and so {{DiscoCache}} history is updated by removing the failed node from the list of alive nodes.
At the same time, 'exchange-worker' detects that there is only one alive node (node 'B' in our case) and mistakenly believes that node 'B' is a coordinator:
{code:java|title=CacheAffinitySharedManager.java}
    void processClientCachesChanges(ClientCacheChangeDummyDiscoveryMessage msg) {
        AffinityTopologyVersion topVer = cctx.exchange().readyAffinityVersion();

        DiscoCache discoCache = cctx.discovery().discoCache(topVer);

        boolean crd = cctx.localNode().equals(discoCache.oldestAliveServerNode()); // discoCache contains only one alive node!

        Map<Integer, Boolean> startedCaches = processClientCacheStartRequests(msg, crd, topVer, discoCache);

        Set<Integer> closedCaches = processCacheCloseRequests(msg, crd, topVer);

        if (startedCaches != null || closedCaches != null)
            scheduleClientChangeMessage(startedCaches, closedCaches);
    }
{code}
and results in the following {{NullPointerException}}:
{code:java}
[19:25:57,019][ERROR][exchange-worker-#42][GridCachePartitionExchangeManager] Failed to process custom exchange task: ClientCacheChangeDummyDiscoveryMessage [reqId=8c7904a2-4b70-4614-bf7b-f4434d274c30, cachesToClose=null, startCaches=[default]]
java.lang.NullPointerException
at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processClientCacheStartRequests(CacheAffinitySharedManager.java:458)
at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processClientCachesChanges(CacheAffinitySharedManager.java:621)
at org.apache.ignite.internal.processors.cache.GridCacheProcessor.processCustomExchangeTask(GridCacheProcessor.java:363)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.processCustomTask(GridCachePartitionExchangeManager.java:2207)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2296)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)
{code}
As a result, the node cannot be stopped due to the following reasons:
 * 'exchange' thread throws {{NullPointerException}} and therefore does not complete {{DynamicCacheStartFuture}}
 * 'Client connector' thread is blocked on {{DynamicCacheStartFuture.get()}} method which never returns control
 * the thread which performs node stopping process is blocked on {{busyLock}}

 Please see the following thread dump:
{code:java}
"Thread-117" #734 prio=5 os_prio=0 tid=0x0000558b117a9000 nid=0x437 waiting on condition [0x00007f2466ba1000]
java.lang.Thread.State: TIMED_WAITING (sleeping)
at java.lang.Thread.sleep(Native Method)
at org.apache.ignite.internal.util.GridSpinReadWriteLock.writeLock(GridSpinReadWriteLock.java:206)
at org.apache.ignite.internal.util.GridSpinBusyLock.block(GridSpinBusyLock.java:76)
at org.apache.ignite.internal.processors.odbc.ClientListenerProcessor.onKernalStop(ClientListenerProcessor.java:224)
at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:2146)
at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:2094)
at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2545)
- locked <0x00000004c0326278> (a org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance)
at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2508)
at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:365)
at org.apache.ignite.Ignition.stop(Ignition.java:229)
at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$10.run(GridDiscoveryManager.java:2385)
at java.lang.Thread.run(Thread.java:748)

"client-connector-#186" #240 prio=5 os_prio=0 tid=0x0000558b12afd000 nid=0x249 waiting on condition [0x00007f246667f000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
at org.apache.ignite.internal.processors.cache.GridCacheProcessor.createMissingQueryCaches(GridCacheProcessor.java:3947)
at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.prepareStatementAndCaches(IgniteH2Indexing.java:2015)
at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.parseAndSplit(IgniteH2Indexing.java:1794)
at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.querySqlFields(IgniteH2Indexing.java:1650)
at org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:2037)
at org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:2032)
at org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
at org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:2580)
at org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2046)
at org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler.executeQuery(JdbcRequestHandler.java:379)
at org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler.handle(JdbcRequestHandler.java:178)
at org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.onMessage(ClientListenerNioListener.java:160)
at org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.onMessage(ClientListenerNioListener.java:44)
at org.apache.ignite.internal.util.nio.GridNioFilterChain$TailFilter.onMessageReceived(GridNioFilterChain.java:279)
at org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:109)
at org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter$3.body(GridNioAsyncNotifyFilter.java:97)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at org.apache.ignite.internal.util.worker.GridWorkerPool$1.run(GridWorkerPool.java:70)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)

"disco-event-worker-#217" #276 prio=5 os_prio=0 tid=0x0000558b10adc000 nid=0x26e waiting on condition [0x00007f2464b3c000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004c00d1350> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body0(GridDiscoveryManager.java:2600)
at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body(GridDiscoveryManager.java:2582)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)

"exchange-worker-#218" #279 prio=5 os_prio=0 tid=0x0000558b10993800 nid=0x271 waiting on condition [0x00007f24624e5000]
java.lang.Thread.State: TIMED_WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004c01782c0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
at java.util.concurrent.LinkedBlockingDeque.pollFirst(LinkedBlockingDeque.java:522)
at java.util.concurrent.LinkedBlockingDeque.poll(LinkedBlockingDeque.java:684)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2290)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)

{code}

  was:
Let's consider the following scenario:
 * Start a new node (node 'A') and create a new partitioned cache that resides on that node

{code:java}
Ignite ignite = Ignition.start("examples/config/segmentation/node-A.xml");
IgniteCache<String, String> cache = ignite.getOrCreateCache(new CacheConfiguration<String, String>()
    .setName("default")
    .setIndexedTypes(String.class, String.class)
    .setNodeFilter(new NodeFilter())
);

public class NodeFilter implements IgnitePredicate<ClusterNode> {
    @Override public boolean apply(ClusterNode node) {
        return node.attribute("test.attribute").equals("first-node");
    }
}{code}
 * Start the second node (node 'B') with a custom connector configuration:

{code:java}
<property name="clientConnectorConfiguration">
    <bean class="org.apache.ignite.configuration.ClientConnectorConfiguration">
        <property name="host" value="127.0.0.1"/>
        <property name="port" value="22222"/>
        <property name="portRange" value="10"/>
    </bean>
</property>

Ignite ignite = Ignition.start("examples/config/segmentation/node-B.xml");

Executors.newScheduledThreadPool(1).schedule(
    new Runnable() {
        @Override public void run() {
            DiscoverySpi spi = ignite.configuration().getDiscoverySpi();
            spi.failNode(ignite.cluster().localNode().id(), "test message");
        }
    },
    30,
    TimeUnit.SECONDS);{code}
 * Execute simple SQL query using sqlline for example (JDBC driver should be connected to the node 'B')

{code:java}
./sqlline.sh --verbose=true -u jdbc:ignite:thin://127.0.0.1:22222

select * from UNKNOWN_TABLE;{code}
In that case, {{IgniteH2Indexing.prepareStatement()}} throws {{SQLException(Table not found)}} and the implementation (see {{IgniteH2Indexing.prepareStatementAndCaches()}}) tries to start caches that are not started yet by sending {{ClientCacheChangeDummyDiscoveryMessage}} to 'discovery-worker' thread,
which in turn posts that message to 'exchange-worker' thread.

Assume that while processing of {{ClientCacheChangeDummyDiscoveryMessage}} by the 'exchange-worker', the discovery thread receives {{EVT_NODE_FAILED}} (as a result of segmentation) and so {{DiscoCache}} history is updated by removing the failed node from the list of alive nodes.
At the same time, 'exchange-worker' detects that there is only one alive node (node 'B' in our case) and mistakenly believes that node 'B' is a coordinator:
{code:java|title=CacheAffinitySharedManager.java}
    void processClientCachesChanges(ClientCacheChangeDummyDiscoveryMessage msg) {
        AffinityTopologyVersion topVer = cctx.exchange().readyAffinityVersion();

        DiscoCache discoCache = cctx.discovery().discoCache(topVer);

        boolean crd = cctx.localNode().equals(discoCache.oldestAliveServerNode()); // discoCache contains only the one node!

        Map<Integer, Boolean> startedCaches = processClientCacheStartRequests(msg, crd, topVer, discoCache);

        Set<Integer> closedCaches = processCacheCloseRequests(msg, crd, topVer);

        if (startedCaches != null || closedCaches != null)
            scheduleClientChangeMessage(startedCaches, closedCaches);
    }
{code}
and results in the following {{NullPointerException}}:
{code:java}
[19:25:57,019][ERROR][exchange-worker-#42][GridCachePartitionExchangeManager] Failed to process custom exchange task: ClientCacheChangeDummyDiscoveryMessage [reqId=8c7904a2-4b70-4614-bf7b-f4434d274c30, cachesToClose=null, startCaches=[default]]
java.lang.NullPointerException
at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processClientCacheStartRequests(CacheAffinitySharedManager.java:458)
at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processClientCachesChanges(CacheAffinitySharedManager.java:621)
at org.apache.ignite.internal.processors.cache.GridCacheProcessor.processCustomExchangeTask(GridCacheProcessor.java:363)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.processCustomTask(GridCachePartitionExchangeManager.java:2207)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2296)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)
{code}
As a result, the node cannot be stopped due to the following reasons:
 * 'exchange' thread throws {{NullPointerException}} and therefore does not complete {{DynamicCacheStartFuture}}
 * 'Client connector' thread is blocked on {{DynamicCacheStartFuture.get()}} method which never returns control
 * the thread which performs node stopping process is blocked on {{busyLock}}

 Please see the following thread dump:
{code:java}
"Thread-117" #734 prio=5 os_prio=0 tid=0x0000558b117a9000 nid=0x437 waiting on condition [0x00007f2466ba1000]
java.lang.Thread.State: TIMED_WAITING (sleeping)
at java.lang.Thread.sleep(Native Method)
at org.apache.ignite.internal.util.GridSpinReadWriteLock.writeLock(GridSpinReadWriteLock.java:206)
at org.apache.ignite.internal.util.GridSpinBusyLock.block(GridSpinBusyLock.java:76)
at org.apache.ignite.internal.processors.odbc.ClientListenerProcessor.onKernalStop(ClientListenerProcessor.java:224)
at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:2146)
at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:2094)
at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2545)
- locked <0x00000004c0326278> (a org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance)
at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2508)
at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:365)
at org.apache.ignite.Ignition.stop(Ignition.java:229)
at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$10.run(GridDiscoveryManager.java:2385)
at java.lang.Thread.run(Thread.java:748)

"client-connector-#186" #240 prio=5 os_prio=0 tid=0x0000558b12afd000 nid=0x249 waiting on condition [0x00007f246667f000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
at org.apache.ignite.internal.processors.cache.GridCacheProcessor.createMissingQueryCaches(GridCacheProcessor.java:3947)
at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.prepareStatementAndCaches(IgniteH2Indexing.java:2015)
at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.parseAndSplit(IgniteH2Indexing.java:1794)
at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.querySqlFields(IgniteH2Indexing.java:1650)
at org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:2037)
at org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:2032)
at org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
at org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:2580)
at org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2046)
at org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler.executeQuery(JdbcRequestHandler.java:379)
at org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler.handle(JdbcRequestHandler.java:178)
at org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.onMessage(ClientListenerNioListener.java:160)
at org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.onMessage(ClientListenerNioListener.java:44)
at org.apache.ignite.internal.util.nio.GridNioFilterChain$TailFilter.onMessageReceived(GridNioFilterChain.java:279)
at org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:109)
at org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter$3.body(GridNioAsyncNotifyFilter.java:97)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at org.apache.ignite.internal.util.worker.GridWorkerPool$1.run(GridWorkerPool.java:70)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)

"disco-event-worker-#217" #276 prio=5 os_prio=0 tid=0x0000558b10adc000 nid=0x26e waiting on condition [0x00007f2464b3c000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004c00d1350> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body0(GridDiscoveryManager.java:2600)
at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body(GridDiscoveryManager.java:2582)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)

"exchange-worker-#218" #279 prio=5 os_prio=0 tid=0x0000558b10993800 nid=0x271 waiting on condition [0x00007f24624e5000]
java.lang.Thread.State: TIMED_WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000004c01782c0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
at java.util.concurrent.LinkedBlockingDeque.pollFirst(LinkedBlockingDeque.java:522)
at java.util.concurrent.LinkedBlockingDeque.poll(LinkedBlockingDeque.java:684)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2290)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)

{code}


> SQL query execution may lead to NullPointerException while node is stopped
> --------------------------------------------------------------------------
>
>                 Key: IGNITE-8689
>                 URL: https://issues.apache.org/jira/browse/IGNITE-8689
>             Project: Ignite
>          Issue Type: Bug
>          Components: general
>    Affects Versions: 2.4
>            Reporter: Vyacheslav Koptilin
>            Priority: Major
>         Attachments: node-A.xml, node-B.xml
>
>
> Let's consider the following scenario:
>  * Start a new node (node 'A') and create a new partitioned cache that resides on that node
> {code:java}
> Ignite ignite = Ignition.start("examples/config/segmentation/node-A.xml");
> IgniteCache<String, String> cache = ignite.getOrCreateCache(new CacheConfiguration<String, String>()
>     .setName("default")
>     .setIndexedTypes(String.class, String.class)
>     .setNodeFilter(new NodeFilter())
> );
> public class NodeFilter implements IgnitePredicate<ClusterNode> {
>     @Override public boolean apply(ClusterNode node) {
>         return node.attribute("test.attribute").equals("first-node");
>     }
> }{code}
>  * Start the second node (node 'B') with a custom connector configuration:
> {code:java}
> <property name="clientConnectorConfiguration">
>     <bean class="org.apache.ignite.configuration.ClientConnectorConfiguration">
>         <property name="host" value="127.0.0.1"/>
>         <property name="port" value="22222"/>
>         <property name="portRange" value="10"/>
>     </bean>
> </property>
> Ignite ignite = Ignition.start("examples/config/segmentation/node-B.xml");
> Executors.newScheduledThreadPool(1).schedule(
>     new Runnable() {
>         @Override public void run() {
>             DiscoverySpi spi = ignite.configuration().getDiscoverySpi();
>             spi.failNode(ignite.cluster().localNode().id(), "test message");
>         }
>     },
>     30,
>     TimeUnit.SECONDS);{code}
>  * Execute simple SQL query using sqlline for example (JDBC driver should be connected to the node 'B')
> {code:java}
> ./sqlline.sh --verbose=true -u jdbc:ignite:thin://127.0.0.1:22222
> select * from UNKNOWN_TABLE;{code}
> In that case, {{IgniteH2Indexing.prepareStatement()}} throws {{SQLException(Table not found)}} and the implementation (see {{IgniteH2Indexing.prepareStatementAndCaches()}}) tries to start caches that are not started yet by sending {{ClientCacheChangeDummyDiscoveryMessage}} to 'discovery-worker' thread,
> which in turn posts that message to 'exchange-worker' thread.
> Assume that while processing of {{ClientCacheChangeDummyDiscoveryMessage}} by the 'exchange-worker', the discovery thread receives {{EVT_NODE_FAILED}} (as a result of segmentation) and so {{DiscoCache}} history is updated by removing the failed node from the list of alive nodes.
> At the same time, 'exchange-worker' detects that there is only one alive node (node 'B' in our case) and mistakenly believes that node 'B' is a coordinator:
> {code:java|title=CacheAffinitySharedManager.java}
>     void processClientCachesChanges(ClientCacheChangeDummyDiscoveryMessage msg) {
>         AffinityTopologyVersion topVer = cctx.exchange().readyAffinityVersion();
>         DiscoCache discoCache = cctx.discovery().discoCache(topVer);
>         boolean crd = cctx.localNode().equals(discoCache.oldestAliveServerNode()); // discoCache contains only one alive node!
>         Map<Integer, Boolean> startedCaches = processClientCacheStartRequests(msg, crd, topVer, discoCache);
>         Set<Integer> closedCaches = processCacheCloseRequests(msg, crd, topVer);
>         if (startedCaches != null || closedCaches != null)
>             scheduleClientChangeMessage(startedCaches, closedCaches);
>     }
> {code}
> and results in the following {{NullPointerException}}:
> {code:java}
> [19:25:57,019][ERROR][exchange-worker-#42][GridCachePartitionExchangeManager] Failed to process custom exchange task: ClientCacheChangeDummyDiscoveryMessage [reqId=8c7904a2-4b70-4614-bf7b-f4434d274c30, cachesToClose=null, startCaches=[default]]
> java.lang.NullPointerException
> at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processClientCacheStartRequests(CacheAffinitySharedManager.java:458)
> at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processClientCachesChanges(CacheAffinitySharedManager.java:621)
> at org.apache.ignite.internal.processors.cache.GridCacheProcessor.processCustomExchangeTask(GridCacheProcessor.java:363)
> at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.processCustomTask(GridCachePartitionExchangeManager.java:2207)
> at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2296)
> at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
> at java.lang.Thread.run(Thread.java:748)
> {code}
> As a result, the node cannot be stopped due to the following reasons:
>  * 'exchange' thread throws {{NullPointerException}} and therefore does not complete {{DynamicCacheStartFuture}}
>  * 'Client connector' thread is blocked on {{DynamicCacheStartFuture.get()}} method which never returns control
>  * the thread which performs node stopping process is blocked on {{busyLock}}
>  Please see the following thread dump:
> {code:java}
> "Thread-117" #734 prio=5 os_prio=0 tid=0x0000558b117a9000 nid=0x437 waiting on condition [0x00007f2466ba1000]
> java.lang.Thread.State: TIMED_WAITING (sleeping)
> at java.lang.Thread.sleep(Native Method)
> at org.apache.ignite.internal.util.GridSpinReadWriteLock.writeLock(GridSpinReadWriteLock.java:206)
> at org.apache.ignite.internal.util.GridSpinBusyLock.block(GridSpinBusyLock.java:76)
> at org.apache.ignite.internal.processors.odbc.ClientListenerProcessor.onKernalStop(ClientListenerProcessor.java:224)
> at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:2146)
> at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:2094)
> at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2545)
> - locked <0x00000004c0326278> (a org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance)
> at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2508)
> at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:365)
> at org.apache.ignite.Ignition.stop(Ignition.java:229)
> at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$10.run(GridDiscoveryManager.java:2385)
> at java.lang.Thread.run(Thread.java:748)
> "client-connector-#186" #240 prio=5 os_prio=0 tid=0x0000558b12afd000 nid=0x249 waiting on condition [0x00007f246667f000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
> at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
> at org.apache.ignite.internal.processors.cache.GridCacheProcessor.createMissingQueryCaches(GridCacheProcessor.java:3947)
> at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.prepareStatementAndCaches(IgniteH2Indexing.java:2015)
> at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.parseAndSplit(IgniteH2Indexing.java:1794)
> at org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.querySqlFields(IgniteH2Indexing.java:1650)
> at org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:2037)
> at org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:2032)
> at org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
> at org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:2580)
> at org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2046)
> at org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler.executeQuery(JdbcRequestHandler.java:379)
> at org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequestHandler.handle(JdbcRequestHandler.java:178)
> at org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.onMessage(ClientListenerNioListener.java:160)
> at org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.onMessage(ClientListenerNioListener.java:44)
> at org.apache.ignite.internal.util.nio.GridNioFilterChain$TailFilter.onMessageReceived(GridNioFilterChain.java:279)
> at org.apache.ignite.internal.util.nio.GridNioFilterAdapter.proceedMessageReceived(GridNioFilterAdapter.java:109)
> at org.apache.ignite.internal.util.nio.GridNioAsyncNotifyFilter$3.body(GridNioAsyncNotifyFilter.java:97)
> at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
> at org.apache.ignite.internal.util.worker.GridWorkerPool$1.run(GridWorkerPool.java:70)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> "disco-event-worker-#217" #276 prio=5 os_prio=0 tid=0x0000558b10adc000 nid=0x26e waiting on condition [0x00007f2464b3c000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00000004c00d1350> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
> at java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
> at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body0(GridDiscoveryManager.java:2600)
> at org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoveryWorker.body(GridDiscoveryManager.java:2582)
> at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
> at java.lang.Thread.run(Thread.java:748)
> "exchange-worker-#218" #279 prio=5 os_prio=0 tid=0x0000558b10993800 nid=0x271 waiting on condition [0x00007f24624e5000]
> java.lang.Thread.State: TIMED_WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00000004c01782c0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
> at java.util.concurrent.LinkedBlockingDeque.pollFirst(LinkedBlockingDeque.java:522)
> at java.util.concurrent.LinkedBlockingDeque.poll(LinkedBlockingDeque.java:684)
> at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2290)
> at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
> at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)