[
https://issues.apache.org/jira/browse/CASSANDRA-16603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17324172#comment-17324172
]
Haoze Wu commented on CASSANDRA-16603:
--------------------------------------
I see. Injecting a shorter delay would also cause the symptom; we used the
15 minutes just for demo purpose, and >15 minutes delay has been observed in
some real production incident, e.g., ZOOKEEPER-2201.
I'm not saying that this issue that I propose is definitely a bug. It
could be a feature. I'm just curious because this injection not only makes its
own thread stuck, but also affects another thread, according to the jstack I
provided.
From our experience, this kind of injection generally only affects one
thread, and can be tolerated by the system itself, transparent to the client.
Therefore, it reminds me of ZOOKEEPER-2201.
ZOOKEEPER-2201 can be reproduced with a similar injection like this issue, and
it is accepted as a bug by developers because it not only makes its own thread
stuck, but also affects some other services that it shouldn't affect.
The injection in this issue is not an "artificial" scenario, because the
I/O can be particularly susceptible to slowness in a VM (like KAFKA-3042) where
a disk I/O operation may need to go through the network (hang for >15min like
ZOOKEEPER-2201). Therefore, it may be a real scenario in real world. The
question here is perhaps whether the system should be affected for as long as
the potentially long delay in commit log add, or there should be some tolerance
like timeout.
We have been analyzing the workflow for a while, and haven't got a final
conclusion. Specifically, after the injection happens, we are still not clear
why we have:
{code:java}
"MigrationStage:1" #69 daemon prio=5 os_prio=0 tid=0x00007f857c6d1b20
nid=0x6ecd waiting on condition [0x00007f8568cef000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for <0x00000005e5e7abd8> (a
com.google.common.util.concurrent.ListenableFutureTask)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
at java.util.concurrent.FutureTask.get(FutureTask.java:191)
at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:438)
at
org.apache.cassandra.schema.SchemaKeyspace.lambda$flush$1(SchemaKeyspace.java:310)
at
org.apache.cassandra.schema.SchemaKeyspace$$Lambda$263/1985363058.accept(Unknown
Source)
at java.lang.Iterable.forEach(Iterable.java:75)
at org.apache.cassandra.schema.SchemaKeyspace.flush(SchemaKeyspace.java:310)
at
org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1398)
- locked <0x0000000611bd39e0> (a java.lang.Class for
org.apache.cassandra.schema.SchemaKeyspace)
at
org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1380)
- locked <0x0000000611bd39e0> (a java.lang.Class for
org.apache.cassandra.schema.SchemaKeyspace)
at
org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51)
at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
at
org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown
Source)
at java.lang.Thread.run(Thread.java:748)
{code}
> Sporadic CQL operation timeout due to unconfigured table
> --------------------------------------------------------
>
> Key: CASSANDRA-16603
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16603
> Project: Cassandra
> Issue Type: Bug
> Components: Local/Commit Log
> Reporter: Haoze Wu
> Priority: Normal
>
> We were doing some systematic testing in Cassandra stable release 3.11.10
> and found the disk I/O delay may cause some issues.
> We start a Cassandra cluster of 3 nodes with the default configuration,
> in node 1, we open the cqlsh shell and type in:
> {code:java}
> create keyspace ycsb WITH REPLICATION = {'class' : 'SimpleStrategy',
> 'replication_factor': 3 };
> USE ycsb;
> create table usertable (
> y_id varchar primary key,
> field0 varchar,
> field1 varchar,
> field2 varchar,
> field3 varchar,
> field4 varchar,
> field5 varchar,
> field6 varchar,
> field7 varchar,
> field8 varchar,
> field9 varchar);
> {code}
> During this process, we inject a single disk I/O delay in node 2’s
> CommitLog#add method (e.g., in Mutation.serializer.serialize).
> {code:java}
> /**
> * Add a Mutation to the commit log. If CDC is enabled, this can fail.
> *
> * @param mutation the Mutation to add to the log
> * @throws WriteTimeoutException
> */
> public CommitLogPosition add(Mutation mutation) throws
> WriteTimeoutException
> {
> assert mutation != null;
> try (DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get())
> {
> Mutation.serializer.serialize(mutation, dob,
> MessagingService.current_version);
> int size = dob.getLength();
> int totalSize = size + ENTRY_OVERHEAD_SIZE;
> if (totalSize > MAX_MUTATION_SIZE)
> {
> throw new IllegalArgumentException(String.format("Mutation of
> %s is too large for the maximum size of %s",
>
> FBUtilities.prettyPrintMemory(totalSize),
>
> FBUtilities.prettyPrintMemory(MAX_MUTATION_SIZE)));
> }
> // ...
> }
> catch (IOException e)
> {
> throw new FSWriteError(e,
> segmentManager.allocatingFrom().getPath());
> }
> }
> {code}
> In the aforementioned cqlsh shell connected to node 1, when it runs the
> command of “create table usertable ...”, it costs more time than normal, and
> finally shows a warning message:
> {code:java}
> Warning: schema version mismatch detected; check the schema versions of your
> nodes in system.local and system.peers.
> {code}
> This behavior seems reasonable because our injection affects node 2.
> When we open another cqlsh shell connecting to node 2, intent to run some
> commands on the created table:
> {code:java}
> USE ycsb;
> {code}
> The cqlsh shell shows an error message:
> {code:java}
> InvalidRequest: Error from server: code=2200 [Invalid query]
> message="unconfigured table t"
> {code}
> It seems reasonable, too, because our injection affects node 2. However,
> when we run a cqlsh shell connecting to node 1 again, and try to run some
> commands like:
> {code:java}
> USE ycsb;
> SELECT * FROM usertable;
> {code}
> Most of the time, the result is shown immediately. However, sometimes it
> gets stuck for a few seconds, and shows the error message:
> {code:java}
> ReadTimeout: Error from server: code=1200 [Coordinator node timed out waiting
> for replica nodes' responses] message="Operation timed out - received only 0
> responses." info={'received_responses': 0, 'required_responses': 1,
> 'consistency': 'ONE'}
> {code}
> A more reliable way to reproduce this failure is using our client based
> on datastax (see [https://github.com/functioner/datastax-cassandra-client]).
> We have dump the jstack for node 2 when the issue happens:
> {code:java}
> "MutationStage-1" #90 daemon prio=5 os_prio=0 tid=0x00007f858429c880
> nid=0x6f23 waiting on condition [0x00007f856889e000]
> java.lang.Thread.State: TIMED_WAITING (sleeping)
> at java.lang.Thread.sleep(Native Method)
> at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:246)
> at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:593)
> at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:470)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:250)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:255)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:264)
> at
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:597)
> at
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:577)
> at
> org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:315)
> at
> org.apache.cassandra.db.SystemKeyspace.lambda$updatePeerInfo$1(SystemKeyspace.java:757)
> at
> org.apache.cassandra.db.SystemKeyspace$$Lambda$286/119781138.run(Unknown
> Source)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:113)
> at java.lang.Thread.run(Thread.java:748)
> "MigrationStage:1" #69 daemon prio=5 os_prio=0 tid=0x00007f857c6d1b20
> nid=0x6ecd waiting on condition [0x00007f8568cef000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00000005e5e7abd8> (a
> com.google.common.util.concurrent.ListenableFutureTask)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
> at java.util.concurrent.FutureTask.get(FutureTask.java:191)
> at
> org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:438)
> at
> org.apache.cassandra.schema.SchemaKeyspace.lambda$flush$1(SchemaKeyspace.java:310)
> at
> org.apache.cassandra.schema.SchemaKeyspace$$Lambda$263/1985363058.accept(Unknown
> Source)
> at java.lang.Iterable.forEach(Iterable.java:75)
> at
> org.apache.cassandra.schema.SchemaKeyspace.flush(SchemaKeyspace.java:310)
> at
> org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1398)
> - locked <0x0000000611bd39e0> (a java.lang.Class for
> org.apache.cassandra.schema.SchemaKeyspace)
> at
> org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1380)
> - locked <0x0000000611bd39e0> (a java.lang.Class for
> org.apache.cassandra.schema.SchemaKeyspace)
> at
> org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51)
> at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:748)
> "MemtablePostFlush:1" #22 daemon prio=5 os_prio=0 tid=0x00007f8598b018e0
> nid=0x6e45 waiting on condition [0x00007f856e1ad000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x00000005e5e7abb8> (a
> java.util.concurrent.CountDownLatch$Sync)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
> at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
> at
> org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:1019)
> at
> org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:1002)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:748)
> "MemtableFlushWriter:2" #19 daemon prio=5 os_prio=0 tid=0x00007f8598af9da0
> nid=0x6e42 waiting on condition [0x00007f856e270000]
> 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.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:280)
> at
> org.apache.cassandra.utils.concurrent.OpOrder$Group.await(OpOrder.java:264)
> at
> org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:431)
> at
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1114)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$6/1239807799.run(Unknown
> Source)
> {code}
> We found that the delay in CommitLog#add make another thread
> (DefinitionsUpdateVerbHandler$1.runMayThrow) stuck. In particular,
> `DefinitionsUpdateVerbHandler` is stuck in waiting for
> `SchemaKeyspace.flush`, which in turn waits for the
> `ColumnFamilyStore.waitForFlushes`.
> We are not sure what is the best place to mitigate the issue. One
> potential fix is to perhaps introduce a timeout limit (default no limit) in
> the add method, e.g., execute the I/O operations in a Future task and call
> waitOnFutures with a timeout in the end. If the timeout is exceeded, a
> WriteTimeoutException would be thrown.
> Any comments and suggestions would be appreciated.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]