Details
-
Bug
-
Status: Open
-
Normal
-
Resolution: Unresolved
-
None
-
None
-
Normal
Description
Some time ago, when our cassandra cluster started, we found that it could not be started, checked various logs, and found no error message. Later, we obtained the thread stack information at startup by some means, and then refer to the source code, and found that if the commitlog initialization error occurs at startup, cassandra will appear deadlock and hang.
The thread stack of COMMIT-LOG-ALLOCATOR:
{ "waitedCount": 0, "lockOwnerId": -1, "lockedMonitors": [], "waitedTime": -1, "blockedCount": 0, "threadState": "RUNNABLE", "inNative": false, "suspended": false, "threadName": "COMMIT-LOG-ALLOCATOR", "lockInfo": null, "threadId": 36, "stackTrace": [ { "fileName": "AbstractCommitLogSegmentManager.java", "nativeMethod": false, "methodName": "runMayThrow", "className": "org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager$1", "lineNumber": 133 }, { "fileName": "WrappedRunnable.java", "nativeMethod": false, "methodName": "run", "className": "org.apache.cassandra.utils.WrappedRunnable", "lineNumber": 28 }, { "fileName": "NamedThreadFactory.java", "nativeMethod": false, "methodName": "lambda$threadLocalDeallocator$0", "className": "org.apache.cassandra.concurrent.NamedThreadFactory", "lineNumber": 81 }, { "fileName": null, "nativeMethod": false, "methodName": "run", "className": "org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4\/1392794732", "lineNumber": -1 }, { "fileName": "Thread.java", "nativeMethod": false, "methodName": "run", "className": "java.lang.Thread", "lineNumber": 748 } ], "blockedTime": -1, "lockName": null, "lockOwnerName": null, "lockedSynchronizers": [] }
The thread stack of main thread:
{ "waitedCount": 2, "lockOwnerId": -1, "lockedMonitors": [ { "identityHashCode": 600118828, "lockedStackDepth": 10, "className": "java.lang.Class", "lockedStackFrame": { "fileName": "ColumnFamilyStore.java", "nativeMethod": false, "methodName": "createColumnFamilyStore", "className": "org.apache.cassandra.db.ColumnFamilyStore", "lineNumber": 620 } }, { "identityHashCode": 600118828, "lockedStackDepth": 11, "className": "java.lang.Class", "lockedStackFrame": { "fileName": "ColumnFamilyStore.java", "nativeMethod": false, "methodName": "createColumnFamilyStore", "className": "org.apache.cassandra.db.ColumnFamilyStore", "lineNumber": 594 } }, { "identityHashCode": 1087037934, "lockedStackDepth": 15, "className": "java.lang.Class", "lockedStackFrame": { "fileName": "Keyspace.java", "nativeMethod": false, "methodName": "open", "className": "org.apache.cassandra.db.Keyspace", "lineNumber": 127 } } ], "waitedTime": -1, "blockedCount": 0, "threadState": "WAITING", "inNative": false, "suspended": false, "threadName": "main", "lockInfo": null, "threadId": 1, "stackTrace": [ { "fileName": "Unsafe.java", "nativeMethod": true, "methodName": "park", "className": "sun.misc.Unsafe", "lineNumber": -2 }, { "fileName": "LockSupport.java", "nativeMethod": false, "methodName": "park", "className": "java.util.concurrent.locks.LockSupport", "lineNumber": 304 }, { "fileName": "WaitQueue.java", "nativeMethod": false, "methodName": "awaitUninterruptibly", "className": "org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal", "lineNumber": 280 }, { "fileName": "AbstractCommitLogSegmentManager.java", "nativeMethod": false, "methodName": "awaitAvailableSegment", "className": "org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager", "lineNumber": 262 }, { "fileName": "AbstractCommitLogSegmentManager.java", "nativeMethod": false, "methodName": "advanceAllocatingFrom", "className": "org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager", "lineNumber": 236 }, { "fileName": "AbstractCommitLogSegmentManager.java", "nativeMethod": false, "methodName": "start", "className": "org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager", "lineNumber": 153 }, { "fileName": "CommitLog.java", "nativeMethod": false, "methodName": "start", "className": "org.apache.cassandra.db.commitlog.CommitLog", "lineNumber": 120 }, { "fileName": "CommitLog.java", "nativeMethod": false, "methodName": "construct", "className": "org.apache.cassandra.db.commitlog.CommitLog", "lineNumber": 93 }, { "fileName": "CommitLog.java", "nativeMethod": false, "methodName": "<clinit>", "className": "org.apache.cassandra.db.commitlog.CommitLog", "lineNumber": 66 }, { "fileName": "ColumnFamilyStore.java", "nativeMethod": false, "methodName": "<init>", "className": "org.apache.cassandra.db.ColumnFamilyStore", "lineNumber": 413 }, { "fileName": "ColumnFamilyStore.java", "nativeMethod": false, "methodName": "createColumnFamilyStore", "className": "org.apache.cassandra.db.ColumnFamilyStore", "lineNumber": 620 }, { "fileName": "ColumnFamilyStore.java", "nativeMethod": false, "methodName": "createColumnFamilyStore", "className": "org.apache.cassandra.db.ColumnFamilyStore", "lineNumber": 594 }, { "fileName": "ColumnFamilyStore.java", "nativeMethod": false, "methodName": "createColumnFamilyStore", "className": "org.apache.cassandra.db.ColumnFamilyStore", "lineNumber": 585 }, { "fileName": "Keyspace.java", "nativeMethod": false, "methodName": "initCf", "className": "org.apache.cassandra.db.Keyspace", "lineNumber": 409 }, { "fileName": "Keyspace.java", "nativeMethod": false, "methodName": "<init>", "className": "org.apache.cassandra.db.Keyspace", "lineNumber": 322 }, { "fileName": "Keyspace.java", "nativeMethod": false, "methodName": "open", "className": "org.apache.cassandra.db.Keyspace", "lineNumber": 127 }, { "fileName": "Keyspace.java", "nativeMethod": false, "methodName": "open", "className": "org.apache.cassandra.db.Keyspace", "lineNumber": 104 }, { "fileName": "SystemKeyspace.java", "nativeMethod": false, "methodName": "checkHealth", "className": "org.apache.cassandra.db.SystemKeyspace", "lineNumber": 935 }, { "fileName": "StartupChecks.java", "nativeMethod": false, "methodName": "execute", "className": "org.apache.cassandra.service.StartupChecks$10", "lineNumber": 382 }, { "fileName": "StartupChecks.java", "nativeMethod": false, "methodName": "verify", "className": "org.apache.cassandra.service.StartupChecks", "lineNumber": 122 }, { "fileName": "CassandraDaemon.java", "nativeMethod": false, "methodName": "setup", "className": "org.apache.cassandra.service.CassandraDaemon", "lineNumber": 200 }, { "fileName": "CassandraDaemon.java", "nativeMethod": false, "methodName": "activate", "className": "org.apache.cassandra.service.CassandraDaemon", "lineNumber": 600 }, { "fileName": "CassandraDaemon.java", "nativeMethod": false, "methodName": "main", "className": "org.apache.cassandra.service.CassandraDaemon", "lineNumber": 689 } ], "blockedTime": -1, "lockName": null, "lockOwnerName": null, "lockedSynchronizers": [] }
The problematic code is in this position.
Since the CommitLog class has not been loaded successfully, the handleCommitError method cannot be called, so that we can't see the error message all the time. CommitLog is loading in the main thread, but it has not been completed, the code location is here.