[jira] [Commented] (CASSANDRA-14380) Cassandra crashes after fsync exception

2018-04-12 Thread Ariel Weisberg (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14380?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16436743#comment-16436743
 ] 

Ariel Weisberg commented on CASSANDRA-14380:


Created CASSANDRA-14383

> Cassandra crashes after fsync exception
> ---
>
> Key: CASSANDRA-14380
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14380
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Adam Geiger
>Priority: Critical
> Attachments: debug.log, debug.log.1.zip, 
> logs-from-cassandra-in-r97bb66e967-apiconnect-cc-0.txt
>
>
> Running Cassandra with a Rook Ceph filesystem within Kubernetes.  During the 
> startup, the following Warnings in the debug log pop up and then Cassandra 
> crashes shortly after and restarts.  It looks like before hitting this error, 
> it is doing a lot of writing and flushing
> WARN [MemtableFlushWriter:2] 2018-04-11 14:34:42,748 NativeLibrary.java:328 - 
> fsync(666) failed, errorno (22) {}
> com.sun.jna.LastErrorException: [22] Invalid argument
>  at org.apache.cassandra.utils.NativeLibraryLinux.fsync(Native Method) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.utils.NativeLibraryLinux.callFsync(NativeLibraryLinux.java:107)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.NativeLibrary.trySync(NativeLibrary.java:317) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.SyncUtil.trySync(SyncUtil.java:179) 
> [apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.SyncUtil.trySyncDir(SyncUtil.java:190) 
> [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.util.SequentialWriter.openChannel(SequentialWriter.java:107)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.util.SequentialWriter.(SequentialWriter.java:141)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.writeMetadata(BigTableWriter.java:402)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.access$300(BigTableWriter.java:53)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter$TransactionalProxy.doPrepare(BigTableWriter.java:368)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.SSTableWriter.prepareToCommit(SSTableWriter.java:281)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.prepareToCommit(SimpleSSTableMultiWriter.java:101)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1153)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1086)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1160)
>  [na:1.8.0]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>  [na:1.8.0]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$12.BCF32600.run(Unknown
>  Source) ~[na:na]
>  at java.lang.Thread.run(Thread.java:811) ~[na:2.9 (12-15-2017)]
>  
> Syslog shows the following 
> (logs-from-cassandra-in-r97bb66e967-apiconnect-cc-0.txt):
> INFO  [main] 2018-04-11 14:49:01,848 ColumnFamilyStore.java:406 - 
> Initializing apim.ur_to_op_by_op
> INFO  [MemoryMXBean notification dispatcher] 2018-04-11 14:49:25,889 
> GCInspector.java:284 - global GC in 206ms.  class storage: 28700680 -> 
> 28692744; miscellaneous non-heap storage: 49871216 -> 53570176; 
> nursery-allocate: 1296878920 -> 149116672; tenured-SOA: 140321968 -> 139143760
> #0: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x302a94) 
> [0x7f17e4f10a94]
> #1: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x306b2d) 
> [0x7f17e4f14b2d]
> #2: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0xc82da) 
> [0x7f17e4cd62da]
> #3: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9prt29.so(+0x22056) 
> [0x7f17e6531056]
> #4: /lib/x86_64-linux-gnu/libpthread.so.0(+0x11390) [0x7f17ed0de390]
> #5: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x2c4e1f) 
> [0x7f17e4ed2e1f]
> #6: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x158c04) 
> [0x7f17e4d66c04]
> #7: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x542d24) 
> [0x7f17e5150d24]
> #8: 

[jira] [Commented] (CASSANDRA-14380) Cassandra crashes after fsync exception

2018-04-12 Thread Jeff Jirsa (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14380?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16436642#comment-16436642
 ] 

Jeff Jirsa commented on CASSANDRA-14380:


[~aweisberg] - worth trying to invoke some special version of an FSError 
handler and/or force a hard/fast shutdown instead of letting it run until it 
crashes? 

> Cassandra crashes after fsync exception
> ---
>
> Key: CASSANDRA-14380
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14380
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Adam Geiger
>Priority: Critical
> Attachments: debug.log, debug.log.1.zip, 
> logs-from-cassandra-in-r97bb66e967-apiconnect-cc-0.txt
>
>
> Running Cassandra with a Rook Ceph filesystem within Kubernetes.  During the 
> startup, the following Warnings in the debug log pop up and then Cassandra 
> crashes shortly after and restarts.  It looks like before hitting this error, 
> it is doing a lot of writing and flushing
> WARN [MemtableFlushWriter:2] 2018-04-11 14:34:42,748 NativeLibrary.java:328 - 
> fsync(666) failed, errorno (22) {}
> com.sun.jna.LastErrorException: [22] Invalid argument
>  at org.apache.cassandra.utils.NativeLibraryLinux.fsync(Native Method) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.utils.NativeLibraryLinux.callFsync(NativeLibraryLinux.java:107)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.NativeLibrary.trySync(NativeLibrary.java:317) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.SyncUtil.trySync(SyncUtil.java:179) 
> [apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.SyncUtil.trySyncDir(SyncUtil.java:190) 
> [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.util.SequentialWriter.openChannel(SequentialWriter.java:107)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.util.SequentialWriter.(SequentialWriter.java:141)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.writeMetadata(BigTableWriter.java:402)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.access$300(BigTableWriter.java:53)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter$TransactionalProxy.doPrepare(BigTableWriter.java:368)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.SSTableWriter.prepareToCommit(SSTableWriter.java:281)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.prepareToCommit(SimpleSSTableMultiWriter.java:101)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1153)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1086)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1160)
>  [na:1.8.0]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>  [na:1.8.0]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$12.BCF32600.run(Unknown
>  Source) ~[na:na]
>  at java.lang.Thread.run(Thread.java:811) ~[na:2.9 (12-15-2017)]
>  
> Syslog shows the following 
> (logs-from-cassandra-in-r97bb66e967-apiconnect-cc-0.txt):
> INFO  [main] 2018-04-11 14:49:01,848 ColumnFamilyStore.java:406 - 
> Initializing apim.ur_to_op_by_op
> INFO  [MemoryMXBean notification dispatcher] 2018-04-11 14:49:25,889 
> GCInspector.java:284 - global GC in 206ms.  class storage: 28700680 -> 
> 28692744; miscellaneous non-heap storage: 49871216 -> 53570176; 
> nursery-allocate: 1296878920 -> 149116672; tenured-SOA: 140321968 -> 139143760
> #0: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x302a94) 
> [0x7f17e4f10a94]
> #1: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x306b2d) 
> [0x7f17e4f14b2d]
> #2: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0xc82da) 
> [0x7f17e4cd62da]
> #3: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9prt29.so(+0x22056) 
> [0x7f17e6531056]
> #4: /lib/x86_64-linux-gnu/libpthread.so.0(+0x11390) [0x7f17ed0de390]
> #5: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x2c4e1f) 
> [0x7f17e4ed2e1f]
> #6: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x158c04) 
> 

[jira] [Commented] (CASSANDRA-14380) Cassandra crashes after fsync exception

2018-04-12 Thread Ariel Weisberg (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-14380?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16436357#comment-16436357
 ] 

Ariel Weisberg commented on CASSANDRA-14380:


This looks like a bad FD, but I just wanted to drop a note here that we can't 
catch fsync errors and continue so we shouldn't add code that does that in C*. 
There was a Postgres bug where fsync returned an error and the FS lost data, 
but subsequent fsyncs succeeded.

The LastErrorException code in NativeLibrary looks a little janky. What's up 
with that? When would trySync be something we would merely try? If try is good 
enough why do it at all considering try is the default behavior of a series of 
unsynced filesystem operations.

> Cassandra crashes after fsync exception
> ---
>
> Key: CASSANDRA-14380
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14380
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Adam Geiger
>Priority: Critical
> Attachments: debug.log, debug.log.1.zip, 
> logs-from-cassandra-in-r97bb66e967-apiconnect-cc-0.txt
>
>
> Running Cassandra with a Rook Ceph filesystem within Kubernetes.  During the 
> startup, the following Warnings in the debug log pop up and then Cassandra 
> crashes shortly after and restarts.  It looks like before hitting this error, 
> it is doing a lot of writing and flushing
> WARN [MemtableFlushWriter:2] 2018-04-11 14:34:42,748 NativeLibrary.java:328 - 
> fsync(666) failed, errorno (22) {}
> com.sun.jna.LastErrorException: [22] Invalid argument
>  at org.apache.cassandra.utils.NativeLibraryLinux.fsync(Native Method) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.utils.NativeLibraryLinux.callFsync(NativeLibraryLinux.java:107)
>  ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.NativeLibrary.trySync(NativeLibrary.java:317) 
> ~[apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.SyncUtil.trySync(SyncUtil.java:179) 
> [apache-cassandra-3.11.0.jar:3.11.0]
>  at org.apache.cassandra.utils.SyncUtil.trySyncDir(SyncUtil.java:190) 
> [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.util.SequentialWriter.openChannel(SequentialWriter.java:107)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.util.SequentialWriter.(SequentialWriter.java:141)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.writeMetadata(BigTableWriter.java:402)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.access$300(BigTableWriter.java:53)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter$TransactionalProxy.doPrepare(BigTableWriter.java:368)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.format.SSTableWriter.prepareToCommit(SSTableWriter.java:281)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.prepareToCommit(SimpleSSTableMultiWriter.java:101)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1153)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1086)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1160)
>  [na:1.8.0]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
>  [na:1.8.0]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.0.jar:3.11.0]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$12.BCF32600.run(Unknown
>  Source) ~[na:na]
>  at java.lang.Thread.run(Thread.java:811) ~[na:2.9 (12-15-2017)]
>  
> Syslog shows the following 
> (logs-from-cassandra-in-r97bb66e967-apiconnect-cc-0.txt):
> INFO  [main] 2018-04-11 14:49:01,848 ColumnFamilyStore.java:406 - 
> Initializing apim.ur_to_op_by_op
> INFO  [MemoryMXBean notification dispatcher] 2018-04-11 14:49:25,889 
> GCInspector.java:284 - global GC in 206ms.  class storage: 28700680 -> 
> 28692744; miscellaneous non-heap storage: 49871216 -> 53570176; 
> nursery-allocate: 1296878920 -> 149116672; tenured-SOA: 140321968 -> 139143760
> #0: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x302a94) 
> [0x7f17e4f10a94]
> #1: /opt/ibm/java/jre/lib/amd64/compressedrefs/libj9jit29.so(+0x306b2d) 
> [0x7f17e4f14b2d]
> #2: