[
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 is
not found) and the implementation 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:
and results in the following NullPointerException:
> SQL query execution may lead to NullPointerException during node stop.
> ----------------------------------------------------------------------
>
> 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
>
> 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
> is not found) and the implementation 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:
> and results in the following NullPointerException:
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)