Details
-
Bug
-
Status: Open
-
Normal
-
Resolution: Unresolved
-
None
-
None
-
Availability
-
Normal
-
Normal
-
Workload Replay
-
All
-
None
Description
We were doing some systematic testing in Cassandra stable release 2.0.3 and we found a transient disk failure(a FileNotFoundException) during flushing in column family creation could lead to the situation that afterwards snapshot repair of that CF blocks forever.
In the workload, we start a cluster of 3 nodes. Then we start a client to the first node to create the table. Afterwards, we start 2 clients each node to do read and write.
The create request to the first node will then be synced to the second node to create the column family, keyspace, and the table at the second node.
However, because of transient disk failure, there could be FileNotFoundException thrown in
in MmappedSegmentedFile#createSegments(String path) during this process:
try { raf = new RandomAccessFile(path, "r"); // Exception here!!! } catch (FileNotFoundException e) { throw new RuntimeException(e); }
In the second node's log:
2023-08-12 16:09:08,927 - ERROR [FlushWriter:1:CassandraDaemon$2@187] - Exception in thread Thread[FlushWriter:1,5,main] java.lang.RuntimeException: java.io.FileNotFoundException: at org.apache.cassandra.io.util.MmappedSegmentedFile$Builder.createSegments(MmappedSegmentedFile.java:183) at org.apache.cassandra.io.util.MmappedSegmentedFile$Builder.complete(MmappedSegmentedFile.java:168) at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SSTableWriter.java:316) at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SSTableWriter.java:306) at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:372) at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:320) at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 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) Caused by: java.io.FileNotFoundException: Inject Error! at org.apache.cassandra.io.util.MmappedSegmentedFile$Builder.createSegments(MmappedSegmentedFile.java:178) ... 10 more
Then the afterwards version request from other nodes would fail because the column family would fail to be created:
2023-08-12 16:09:34,931 - WARN [Thread-12:IncomingTcpConnection@83] - UnknownColumnFamilyException reading from socket; closing
org.apache.cassandra.db.UnknownColumnFamilyException: Couldn't find cfId=7f032c76-8e13-34ff-8d56-24fa66dcb6ff
at org.apache.cassandra.db.ColumnFamilySerializer.deserializeCfId(ColumnFamilySerializer.java:178)
at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:103)
at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserializeOneCf(RowMutation.java:304)
at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:284)
at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:312)
at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:254)
at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99)
at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:153)
at org.apache.cassandra.net.IncomingTcpConnection.handleModernVersion(IncomingTcpConnection.java:130)
at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74)
Also, the afterwards read/write request would fail because keyspace not exist:
2023-08-12 16:09:49,333 - ERROR [ReadStage:11:CassandraDaemon$2@187] - Exception in thread Thread[ReadStage:11,5,main] java.lang.AssertionError: Unknown keyspace gray_space at org.apache.cassandra.db.Keyspace.<init>(Keyspace.java:262) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:110) at org.apache.cassandra.db.Keyspace.open(Keyspace.java:88) at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:43) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) 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)
Also, if we initiate a repair request at node1, we will encounter the situation in Cassandra-6415, which is that node1 will block in makeSnapshots forever:
"AntiEntropySessions:1" #458 daemon prio=5 os_prio=0 tid=0x00007fa954032800 nid=0x64c4 waiting on condition [0x00007fa6d218e000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x000000062464e1a0> (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.repair.RepairJob.makeSnapshots(RepairJob.java:140) at org.apache.cassandra.repair.RepairJob.sendTreeRequests(RepairJob.java:109) at org.apache.cassandra.repair.RepairSession.runMayThrow(RepairSession.java:267) 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 java.lang.Thread.run(Thread.java:748)
Although in Cassandra-6455, we propose a longer term solution for the block. However, it
only targets for the symptom. As to the root cause in this case, it looks that at node1 and node3, the node2 is considered as a valid companion even the table, keyspace, and columnfamily does not exist on node2. This leads to a persistent inconsistency.
As to the root cause, one potential fix is to recovery from this by considering recreating the columnfamily after the failure. Or maybe other nodes would not consider the failed node as a valid companion for this columnfamily and do not consider it in afterwards activity.
We propose a valid way to reproduce it in this repo: https://github.com/tonyPan123/cassandra-18748
Any comments and suggestions would be appreciated.
Attachments
Issue Links
- is related to
-
CASSANDRA-6415 Snapshot repair blocks for ever if something happens to the "I made my snapshot" response
- Resolved