Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
Description
The following test hangs intermittently (once for 4-5 runs) on my laptop (Ubuntu 20.04, i7-8565u, 16gb RAM, JDK 1.8.0_251). The cursor iteration randomly hangs on the stage of waiting for the next page from the remote node.
/** */ public static final int NODES_CNT = 2; /** */ public static final int TABLE_POPULATION = 2000; /** */ public static final int SELECT_RANGE = 1000; /** */ public static final int QRY_PAGE_SIZE = 5; /** */ @Test public void test() throws Exception { for (int i = 0; i < NODES_CNT; i++) startGrid(i, false); IgniteEx cli = startGrid(NODES_CNT, true); GridQueryProcessor qryProc = cli.context().query(); qryProc.querySqlFields( new SqlFieldsQuery("CREATE TABLE test_table (id LONG PRIMARY KEY, val LONG)"), false); qryProc.querySqlFields(new SqlFieldsQuery("CREATE INDEX val_idx ON test_table (val)"), false); for (long l = 0; l < TABLE_POPULATION; ++l) { qryProc.querySqlFields( new SqlFieldsQuery("INSERT INTO test_table (id, val) VALUES (?, ?)").setArgs(l, l), true ); } for (int i = 0; i < 10000 ; i++) { long lowId = ThreadLocalRandom.current().nextLong(TABLE_POPULATION - SELECT_RANGE); long highId = lowId + SELECT_RANGE; try ( FieldsQueryCursor<List<?>> cursor = cli .context().query().querySqlFields( new SqlFieldsQuery("SELECT id, val FROM test_table WHERE id BETWEEN ? and ?") .setArgs(lowId, highId) .setPageSize(QRY_PAGE_SIZE), false ) ) { cursor.iterator().forEachRemaining(val -> {}); } } } /** */ private IgniteEx startGrid(int idx, boolean clientMode) throws Exception { return (IgniteEx) Ignition.start(new IgniteConfiguration() .setIgniteInstanceName("node-" + idx) .setGridLogger(new Log4JLogger("modules/core/src/test/config/log4j-test.xml")) .setClientMode(clientMode)); }
UPD It seems that IGNITE-12845 is responsible for the behavior described above. Commit which is related to this ticket is the first since which the code mentioned above started to hang.
Cursor iteration hangs due to GridQueryNextPageRequest in some cases are not sent correctly from the client node.
UPD Simplified reproducer of the problem described above:
@Test public void test() throws Exception { IgniteEx srv = startGrid(0); IgniteEx cli = startClientGrid(1); GridQueryNextPageRequest msg = new GridQueryNextPageRequest(0, 0, 0, 0, (byte)0); CyclicBarrier barrier = new CyclicBarrier(2); srv.context().io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() { @Override public void onMessage(UUID nodeId, Object msg, byte plc) { try { if (msg instanceof GridQueryNextPageRequest) barrier.await(); } catch (InterruptedException | BrokenBarrierException e) { throw new RuntimeException(e); } } }); for (int i = 0; i < 1000; i++) { barrier.reset(); cli.context().io().sendToGridTopic(srv.context().discovery().localNode(), GridTopic.TOPIC_QUERY, msg, GridIoPolicy.QUERY_POOL); try { barrier.await(1, TimeUnit.SECONDS); } catch (InterruptedException | BrokenBarrierException | TimeoutException e) { fail(); } } }
The root cause of the hanging is lack of synchronization between org.apache.ignite.internal.util.nio.GridNioServer#stopPollingForWrite and org.apache.ignite.internal.util.nio.GridNioServer#send0 methods. The following situation is possible:
1. In stopPollingForWrite method worker thread checks that the the queue is empty:
if (ses.writeQueue().isEmpty()) {
and this condition appears true. The worker thread stops its execution.
2. Message sender thread calls send0 method and it returns. org.apache.ignite.internal.util.nio.GridSelectorNioSessionImpl#procWrite was not set to false yet, so sending message isn't added to worker queue due to:
else if (!ses.procWrite.get() && ses.procWrite.compareAndSet(false, true)) {
3. The worker thread continues stopPollingForWrite execution and disables OP_WRITE flag, which means that socket write events are no longer listened.
So the message remains unsent.
Attachments
Issue Links
- is caused by
-
IGNITE-12845 GridNioServer can infinitely lose some events
- Resolved
- links to