You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Philip Ó Condúin <ph...@gmail.com> on 2019/08/07 10:46:00 UTC

Datafile Corruption

Hi All,

I am currently experiencing multiple datafile corruptions across most nodes
in my cluster, there seems to be no pattern to the corruption.  I'm
starting to think it might be a bug, we're using Cassandra 2.2.13.

Without going into detail about the issue I just want to confirm something.

Can someone share with me a list of scenarios that would cause corruption?

1. OS failure
2. Cassandra disturbed during the writing

etc etc.

I need to investigate each scenario and don't want to leave any out.

-- 
Regards,
Phil

Re: Datafile Corruption

Posted by Laxmikant Upadhyay <la...@gmail.com>.
Few for reasons:
Sudden Power cut
Disk full
Issue in casandra version like Cassandra-13752

On Wed, Aug 7, 2019, 4:16 PM Philip Ó Condúin <ph...@gmail.com>
wrote:

> Hi All,
>
> I am currently experiencing multiple datafile corruptions across most
> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
> starting to think it might be a bug, we're using Cassandra 2.2.13.
>
> Without going into detail about the issue I just want to confirm something.
>
> Can someone share with me a list of scenarios that would cause corruption?
>
> 1. OS failure
> 2. Cassandra disturbed during the writing
>
> etc etc.
>
> I need to investigate each scenario and don't want to leave any out.
>
> --
> Regards,
> Phil
>

Re: Datafile Corruption

Posted by Philip Ó Condúin <ph...@gmail.com>.
Confirmed, we are not using NVMe



So, this is my interpretation of the error of one of the corrupt files,
full error stack below;



- original exception is this:

Caused by: org.apache.cassandra.io.compress.CorruptBlockException:
(/data/ssd2/data/KeyspaceMetadata/CF_ToIndex/lb-26203-big-Data.db):
corruption detected, chunk at 1173600152 of length 20802.

at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
~[apache-cassandra-2.2.13.jar:2.2.13]



- it was thrown by this code:

https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
<https://office365.us.vadesecure.com/safeproxy/v3?f=NOi6AJfP03KNFA1AXAD_OpIptbev4O6TYFZCMYvY6alQyrF0EEJcYy6rKZ_ImbTP&i=iUYZ7Fnr_lH_aTE2YZ_3dM8g-hQfE0ZyFKNbv575b7ME0kL72KNwbIsIGgF1DCOmqFbKlJez6AFBNbr5IFzoow&k=e5el&r=AzXKx5Huc_jadBWRpGXHRxiDNFFin3vFKhNB5VXq83a-M3YBpfE9FTmAqKfp-isk&u=https%3A%2F%2Fgithub.com%2Fapache%2Fcassandra%2Fblob%2Fcassandra-2.2%2Fsrc%2Fjava%2Forg%2Fapache%2Fcassandra%2Fio%2Fcompress%2FCompressedRandomAccessReader.java>

line 185



if (metadata.parameters.getCrcCheckChance() >
ThreadLocalRandom.current().nextDouble())

            {

                FBUtilities.directCheckSum(checksum, compressedChunk);



                compressedChunk.limit(compressedChunk.capacity());

                if (compressedChunk.getInt() != (int) checksum.getValue())

*                    throw new CorruptBlockException(getPath(), chunk);*



                // reset checksum object back to the original (blank) state

                checksum.reset();



                compressedChunk.position(chunkOffset).limit(chunkOffset +
chunk.length);

            }



Then it was caught by following code and re-thrown,….



This code might be checking if it should check the CRC (it is controlled by
parameter crc_check_chance which is defined per table and default = 1 which
means always check), this functionality applies only when compression is
used.

If CRC check should happen, then it calls procedure to calculate checksum
from the chunk and compares it with the value stored in file with chunk. If
it does not match – throws an exception.


I wonder could there be a problem with the CRC check?






Full error stack:


INFO  21:30:33 Writing
Memtable-compactions_in_progress@830377457(0.008KiB serialized
bytes, 1 ops, 0%/0% of on/off-heap limit)

ERROR 21:30:33 Failed creating a merkle tree for [repair
#9587a200-b95a-11e9-8920-9f72868b8375 on KeyspaceMetadata/CF_ToIndex,
(-1476350953672479093,-147446147798786

4359]], /10.2.41.56 (see log for details)

ERROR 21:30:33 Exception in thread Thread[ValidationExecutor:825,1,main]

org.apache.cassandra.io.FSReadError:
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
/data/ssd2/data/KeyspaceMetadata/CF_ToIndex-1e453cb09c7911e8ac12255de1fb512a/lb-26203-big-Data.db

at
org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
~[apache-cassandra-2.2.13.jar:2.2.13]

at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
~[apache-cassandra-2.2.13.jar:2.2.13]

at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
~[apache-cassandra-2.2.13.jar:2.2.13]

at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
~[guava-16.0.jar:na]

at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
~[guava-16.0.jar:na]

at
org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
~[guava-16.0.jar:na]

at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
~[guava-16.0.jar:na]

at com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
~[guava-16.0.jar:na]

at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
~[guava-16.0.jar:na]

at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
~[guava-16.0.jar:na]

at
org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
~[apache-cassandra-2.2.13.jar:2.2.13]

at org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
~[apache-cassandra-2.2.13.jar:2.2.13]

at org.apache.cassandra.repair.Validator.add(Validator.java:150)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
~[apache-cassandra-2.2.13.jar:2.2.13]

at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]

at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
~[na:1.8.0_172]

at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[na:1.8.0_172]

at java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]

Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException:
Corrupted: /data/ssd2/data/KeyspaceMetadata/CF_ToIndex/lb-26203-big-Data.db

at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
~[apache-cassandra-2.2.13.jar:2.2.13]

at
org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
~[apache-cassandra-2.2.13.jar:2.2.13]

... 27 common frames omitted

Caused by: org.apache.cassandra.io.compress.CorruptBlockException:
(/data/ssd2/data/KeyspaceMetadata/CF_ToIndex/lb-26203-big-Data.db):
corruption detected, chunk at 1173600152 of length 20802.

at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
~[apache-cassandra-2.2.13.jar:2.2.13]

... 30 common frames omitted

INFO  21:30:33 Not a global repair, will not do anticompaction

ERROR 21:30:33 Stopping gossiper

WARN  21:30:33 Stopping gossip by operator request

INFO  21:30:33 Announcing shutdown

INFO  21:30:33 Node /x.x.x.x state jump to shutdown

INFO  21:30:34 [Stream #961933a1-b95a-11e9-b642-255c22db0481] Session with /
10.2.57.48 is complete

INFO  21:30:34 [Stream #961933a1-b95a-11e9-b642-255c22db0481] All sessions
completed

INFO  21:30:34 [Stream #96190c90-b95a-11e9-8a18-dbd9268d5b6a] Session with /
10.2.57.47 is complete

INFO  21:30:34 [Stream #96190c90-b95a-11e9-8a18-dbd9268d5b6a] All sessions
completed

INFO  21:30:34 [repair #9587a200-b95a-11e9-8920-9f72868b8375] streaming
task succeed, returning response to /x.x.x.x

INFO  21:30:34 [repair #9587a200-b95a-11e9-8920-9f72868b8375] Sending
completed merkle tree to /x.x.x.x for KeyspaceMetadata.CF_CcIndex

ERROR 21:30:35 Stopping RPC server

INFO  21:30:35 Stop listening to thrift clients

ERROR 21:30:35 Stopping native transport

INFO  21:30:35 Stop listening for CQL clients



On Thu, 8 Aug 2019 at 17:58, Philip Ó Condúin <ph...@gmail.com>
wrote:

> Hi Jon,
>
> Good question, I'm not sure if we're using NVMe, I don't see /dev/nvme but
> we could still be using it.
> We using *Cisco UCS C220 M4 SFF* so I'm just going to check the spec.
>
> Our Kernal is the following, we're using REDHAT so I'm told we can't
> upgrade the version until the next major release anyway.
> root@cass 0 17:32:28 ~ # uname -r
> 3.10.0-957.5.1.el7.x86_64
>
> Cheers,
> Phil
>
> On Thu, 8 Aug 2019 at 17:35, Jon Haddad <jo...@jonhaddad.com> wrote:
>
>> Any chance you're using NVMe with an older Linux kernel?  I've seen a
>> *lot* filesystem errors from using older CentOS versions.  You'll want to
>> be using a version > 4.15.
>>
>> On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <ph...@gmail.com>
>> wrote:
>>
>>> *@Jeff *- If it was hardware that would explain it all, but do you
>>> think it's possible to have every server in the cluster with a hardware
>>> issue?
>>> The data is sensitive and the customer would lose their mind if I sent
>>> it off-site which is a pity cause I could really do with the help.
>>> The corruption is occurring irregularly on every server and instance and
>>> column family in the cluster.  Out of 72 instances, we are getting maybe 10
>>> corrupt files per day.
>>> We are using vnodes (256) and it is happening in both DC's
>>>
>>> *@Asad *- internode compression is set to ALL on every server.  I have
>>> checked the packets for the private interconnect and I can't see any
>>> dropped packets, there are dropped packets for other interfaces, but not
>>> for the private ones, I will get the network team to double-check this.
>>> The corruption is only on the application schema, we are not getting
>>> corruption on any system or cass keyspaces.  Corruption is happening in
>>> both DC's.  We are getting corruption for the 1 application schema we have
>>> across all tables in the keyspace, it's not limited to one table.
>>> Im not sure why the app team decided to not use default compression, I
>>> must ask them.
>>>
>>>
>>>
>>> I have been checking the /var/log/messages today going back a few weeks
>>> and can see a serious amount of broken pipe errors across all servers and
>>> instances.
>>> Here is a snippet from one server but most pipe errors are similar:
>>>
>>> Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
>>> Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072
>>> ops, 0%/0% of on/off-heap limit)
>>> Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during
>>> write!
>>> Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>>> ~[libthrift-0.9.2.jar:0.9.2]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during
>>> write!
>>> Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>>> ~[libthrift-0.9.2.jar:0.9.2]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>>
>>>
>>>
>>> On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:
>>>
>>>> Did you check if packets are NOT being dropped for network interfaces
>>>> Cassandra instances are consuming (ifconfig –a) internode compression is
>>>> set for all endpoint – may be network is playing any role here?
>>>>
>>>> is this corruption limited so certain keyspace/table | DCs or is that
>>>> wide spread – the log snippet you shared it looked like only specific
>>>> keyspace/table is affected – is that correct?
>>>>
>>>> When you remove corrupted sstable of a certain table, I guess you
>>>> verifies all nodes for corrupted sstables for same table (may be with with
>>>> nodetool scrub tool) so to limit spread of corruptions – right?
>>>>
>>>> Just curious to know – you’re not using lz4/default compressor for all
>>>> tables there must be some reason for it.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
>>>> *Sent:* Thursday, August 08, 2019 6:20 AM
>>>> *To:* user@cassandra.apache.org
>>>> *Subject:* Re: Datafile Corruption
>>>>
>>>>
>>>>
>>>> Hi All,
>>>>
>>>> Thank you so much for the replies.
>>>>
>>>> Currently, I have the following list that can potentially cause some
>>>> sort of corruption in a Cassandra cluster.
>>>>
>>>>    - Sudden Power cut  -  *We have had no power cuts in the
>>>>    datacenters*
>>>>    - Network Issues - *no network issues from what I can tell*
>>>>    - Disk full - *I don't think this is an issue for us, see disks
>>>>    below.*
>>>>    - An issue in Casandra version like Cassandra-13752 -* couldn't
>>>>    find any Jira issues similar to ours.*
>>>>    - Bit Flips -* we have compression enabled so I don't think this
>>>>    should be an issue.*
>>>>    - Repair during upgrade has caused corruption too -* we have not
>>>>    upgraded*
>>>>    - Dropping and adding columns with the same name but a different
>>>>    type - *I will need to ask the apps team how they are using the
>>>>    database.*
>>>>
>>>>
>>>>
>>>> Ok, let me try and explain the issue we are having, I am under a lot of
>>>> pressure from above to get this fixed and I can't figure it out.
>>>>
>>>> This is a PRE-PROD environment.
>>>>
>>>>    - 2 datacenters.
>>>>    - 9 physical servers in each datacenter
>>>>    - 4 Cassandra instances on each server
>>>>    - 72 Cassandra instances across the 2 data centres, 36 in site A,
>>>>    36 in site B.
>>>>
>>>>
>>>> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
>>>> datacenter each running with its own Cassandra back end in a cluster
>>>> together.
>>>>
>>>> OS Details [Red Hat Linux]
>>>> cass_a@x 0 10:53:01 ~ $ uname -a
>>>> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
>>>> x86_64 x86_64 x86_64 GNU/Linux
>>>>
>>>> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
>>>> NAME="Red Hat Enterprise Linux Server"
>>>> VERSION="7.6 (Maipo)"
>>>> ID="rhel"
>>>>
>>>> Storage Layout
>>>> cass_a@xx 0 10:46:28 ~ $ df -h
>>>> Filesystem                         Size  Used Avail Use% Mounted on
>>>> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
>>>> devtmpfs                            63G     0   63G   0% /dev
>>>> tmpfs                               63G     0   63G   0% /dev/shm
>>>> tmpfs                               63G  4.1G   59G   7% /run
>>>> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>>>> >> 4 cassandra instances
>>>> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
>>>> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
>>>> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
>>>> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>>>>
>>>> Cassandra load is about 200GB and the rest of the space is snapshots
>>>>
>>>> CPU
>>>> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E
>>>> '^Thread|^Core|^Socket|^CPU\('
>>>> CPU(s):                64
>>>> Thread(s) per core:    2
>>>> Core(s) per socket:    16
>>>> Socket(s):             2
>>>>
>>>> *Description of problem:*
>>>> During repair of the cluster, we are seeing multiple corruptions in the
>>>> log files on a lot of instances.  There seems to be no pattern to the
>>>> corruption.  It seems that the repair job is finding all the corrupted
>>>> files for us.  The repair will hang on the node where the corrupted file is
>>>> found.  To fix this we remove/rename the datafile and bounce the Cassandra
>>>> instance.  Our hardware/OS team have stated there is no problem on their
>>>> side.  I do not believe it the repair causing the corruption.
>>>>
>>>> We have maintenance scripts that run every night running compactions
>>>> and creating snapshots, I decided to turn these off, fix any corruptions we
>>>> currently had and ran major compaction on all nodes, once this was done we
>>>> had a "clean" cluster and we left the cluster for a few days.  After the
>>>> process we noticed one corruption in the cluster, this datafile was created
>>>> after I turned off the maintenance scripts so my theory of the scripts
>>>> causing the issue was wrong.  We then kicked off another repair and started
>>>> to find more corrupt files created after the maintenance script was turned
>>>> off.
>>>>
>>>>
>>>> So let me give you an example of a corrupted file and maybe someone
>>>> might be able to work through it with me?
>>>>
>>>> When this corrupted file was reported in the log it looks like it was
>>>> the repair that found it.
>>>>
>>>> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
>>>> --until "2019-08-07 22:45:00"
>>>>
>>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
>>>> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes,
>>>> 1 ops, 0%/0% of on/off-heap limit)
>>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a
>>>> merkle tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
>>>> KeyspaceMetadata/x, (-1476350953672479093,-1474461
>>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
>>>> Thread[ValidationExecutor:825,1,main]
>>>> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
>>>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>>>> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>>> ~[guava-16.0.jar:na]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.repair.Validator.add(Validator.java:150)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>>> ~[na:1.8.0_172]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>>> [na:1.8.0_172]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
>>>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>>>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>>>> /data/ssd2/data/KeyspaceMetadata/x-x/l
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
>>>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>>>> org.apache.cassandra.io.compress.CorruptBlockException:
>>>> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
>>>> Aug 07 22:30:33 cassandra[34611]: at
>>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
>>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>>> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
>>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair,
>>>> will not do anticompaction
>>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
>>>> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
>>>> operator request
>>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
>>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
>>>> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
>>>> state jump to shutdown
>>>>
>>>>
>>>> So I went to the file system to see when this corrupt file was created
>>>> and it was created on July 30th at 15.55
>>>>
>>>> root@x 0 01:14:03 ~ # ls -l
>>>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>>> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55
>>>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>>>
>>>>
>>>>
>>>> So I checked /var/log/messages for errors around that time
>>>> The only thing that stands out to me is the message "Cannot perform a
>>>> full major compaction as repaired and unrepaired sstables cannot be
>>>> compacted together", I'm not sure if this would be an issue though and
>>>> cause corruption.
>>>>
>>>> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
>>>> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
>>>> Jul 30 15:55:06 x audispd: node=x. type=USER_START
>>>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>>>> ses=4294967295 msg='op=PAM:session_open
>>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>>> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
>>>> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
>>>> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
>>>> ses=4294967295 msg='cwd="/"
>>>> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
>>>> terminal=? res=success'
>>>> Jul 30 15:55:14 x tag_audit_log: type=USER_START
>>>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>>>> ses=4294967295 msg='op=PAM:session_open
>>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>>>> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes,
>>>> 1 ops, 0%/0% of on/off-heap limit)
>>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
>>>> compaction as repaired and unrepaired sstables cannot be compacted
>>>> together. These two set of sstables will be compacted separately.
>>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>>>> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes,
>>>> 57 ops, 0%/0% of on/off-heap limit)
>>>> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
>>>> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes,
>>>> 1 ops, 0%/0% of on/off-heap limit)
>>>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>>>> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
>>>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>>>> terminal=/dev/pts/0 res=success'
>>>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>>>> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
>>>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>>>> terminal=/dev/pts/1 res=success'
>>>>
>>>> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
>>>> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
>>>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>>>> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
>>>> ses=4294967295 msg='op=PAM:session_open
>>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>>>> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
>>>> ses=4294967295 msg='op=PAM:session_open
>>>> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
>>>> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
>>>> res=success'
>>>> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
>>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>>>> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes,
>>>> 1 ops, 0%/0% of on/off-heap limit)
>>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
>>>> compaction as repaired and unrepaired sstables cannot be compacted
>>>> together. These two set of sstables will be compacted separately.
>>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>>>> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes,
>>>> 16 ops, 0%/0% of on/off-heap limit)
>>>> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
>>>> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
>>>> ses=4294967295 msg='cwd="/"
>>>> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
>>>> terminal=? res=success'
>>>>
>>>>
>>>>
>>>> We have checked a number of other things like NTP setting etc but
>>>> nothing is telling us what could cause so many corruptions across the
>>>> entire cluster.
>>>> Things were healthy with this cluster for months, the only thing I can
>>>> think is that we started loading data from a load of 20GB per instance up
>>>> to 200GB where it sits now, maybe this just highlighted the issue.
>>>>
>>>>
>>>>
>>>> Compaction and Compression on Keyspace CL's [mixture]
>>>> All CF's are using compression.
>>>>
>>>> AND compaction = {'min_threshold': '4', 'class':
>>>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>>>> 'max_threshold': '32'}
>>>> AND compression = {'sstable_compression':
>>>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>>>
>>>> AND compaction = {'min_threshold': '4', 'class':
>>>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>>>> 'max_threshold': '32'}
>>>> AND compression = {'sstable_compression':
>>>> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>>>>
>>>> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
>>>> *LeveledCompactionStrategy*'}
>>>> AND compression = {'sstable_compression':
>>>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>>>
>>>> --We are also using internode network compression:
>>>> internode_compression: all
>>>>
>>>>
>>>>
>>>> Does anyone have any idea what I should check next?
>>>> Our next theory is that there may be an issue with Checksum, but I'm
>>>> not sure where to go with this.
>>>>
>>>>
>>>>
>>>> Any help would be very much appreciated before I lose the last bit of
>>>> hair I have on my head.
>>>>
>>>>
>>>>
>>>> Kind Regards,
>>>>
>>>> Phil
>>>>
>>>>
>>>>
>>>> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com>
>>>> wrote:
>>>>
>>>> Repair during upgrade have caused corruption too.
>>>>
>>>>
>>>>
>>>> Also, dropping and adding columns with same name but different type
>>>>
>>>>
>>>>
>>>> Regards,
>>>>
>>>> Nitan
>>>>
>>>> Cell: 510 449 9629
>>>>
>>>>
>>>> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>>>>
>>>> Is compression enabled?
>>>>
>>>>
>>>>
>>>> If not, bit flips on disk can corrupt data files and reads + repair may
>>>> send that corruption to other hosts in the cluster
>>>>
>>>>
>>>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
>>>> wrote:
>>>>
>>>> Hi All,
>>>>
>>>>
>>>>
>>>> I am currently experiencing multiple datafile corruptions across most
>>>> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
>>>> starting to think it might be a bug, we're using Cassandra 2.2.13.
>>>>
>>>>
>>>>
>>>> Without going into detail about the issue I just want to confirm
>>>> something.
>>>>
>>>>
>>>>
>>>> Can someone share with me a list of scenarios that would cause
>>>> corruption?
>>>>
>>>>
>>>>
>>>> 1. OS failure
>>>>
>>>> 2. Cassandra disturbed during the writing
>>>>
>>>>
>>>>
>>>> etc etc.
>>>>
>>>>
>>>>
>>>> I need to investigate each scenario and don't want to leave any out.
>>>>
>>>>
>>>>
>>>> --
>>>>
>>>> Regards,
>>>>
>>>> Phil
>>>>
>>>>
>>>>
>>>>
>>>> --
>>>>
>>>> Regards,
>>>>
>>>> Phil
>>>>
>>>
>>>
>>> --
>>> Regards,
>>> Phil
>>>
>>
>
> --
> Regards,
> Phil
>


-- 
Regards,
Phil

Re: Datafile Corruption

Posted by Laxmikant Upadhyay <la...@gmail.com>.
Cool ! Thanks for sharing the RCA.

On Wed, Oct 9, 2019 at 2:56 PM Philip Ó Condúin <ph...@gmail.com>
wrote:

> Just to follow up on this issue as others may see it in the future, we
> cracked it!
>
> Ou datafile corruption issues were a problem with the OS wrongly taking
> one block belonging to a C* data file thinking it was no longer used and
> treating it as a free block that would later be used.
>
> For example:
> C* deletes file after compaction, OS collects all blocks which are free
> now and sends TRIM command to SSD, but SSD from time to time picks the
> wrong block, not the one reported by OS - does the trim - causing zeroized
> blocks to be seen in the datafile and later use it for different file.
> So the symptom is - we suddenly see 4096 zeroes in the datafile- it means
> SSD just trimmed the block, after some time we can see some data written to
> those blocks - it means the block is used by other file and therefore gives
> us a corrupt file.
>
> We turned off the scheduled TRIM function on the OS and we are no longer
> getting corruptions.
>
> This was very difficult to pinpoint.
>
> On Thu, 15 Aug 2019 at 00:09, Patrick McFadin <pm...@gmail.com> wrote:
>
>> If you hadn't mentioned the fact you are using physical disk I would have
>> guessed you were using virtual disks on a SAN. I've seen this sort of thing
>> happen a lot there. Are there any virtual layers between the cassandra
>> process and the hardware? Just a reminder, fsync can be a liar and the
>> virtual layer can mock the response back to user land while the actual bits
>> can be dropped before hitting the disk.
>>
>> If not, you should be looking hard at your disk options. fstab,
>> schedulers, etc. In that case, you need this:
>> https://tobert.github.io/pages/als-cassandra-21-tuning-guide.html
>>
>>
>> Patrick
>>
>> On Wed, Aug 14, 2019 at 2:03 PM Forkalsrud, Erik <ef...@cj.com>
>> wrote:
>>
>>> The dmesg command will usually show information about hardware errors.
>>>
>>> An example from a spinning disk:
>>>     sd 0:0:10:0: [sdi] Unhandled sense code
>>>     sd 0:0:10:0: [sdi] Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
>>>     sd 0:0:10:0: [sdi] Sense Key : Medium Error [current]
>>>     Info fld=0x6fc72
>>>     sd 0:0:10:0: [sdi] Add. Sense: Unrecovered read error
>>>     sd 0:0:10:0: [sdi] CDB: Read(10): 28 00 00 06 fc 70 00 00 08 00
>>>
>>>
>>> Also, you can read the file like
>>> "cat  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db >
>>> /dev/null"
>>> If you get an error message, it's probably a hardware issue.
>>>
>>> - Erik -
>>>
>>> ------------------------------
>>> *From:* Philip Ó Condúin <ph...@gmail.com>
>>> *Sent:* Thursday, August 8, 2019 09:58
>>> *To:* user@cassandra.apache.org <us...@cassandra.apache.org>
>>> *Subject:* Re: Datafile Corruption
>>>
>>> Hi Jon,
>>>
>>> Good question, I'm not sure if we're using NVMe, I don't see /dev/nvme
>>> but we could still be using it.
>>> We using *Cisco UCS C220 M4 SFF* so I'm just going to check the spec.
>>>
>>> Our Kernal is the following, we're using REDHAT so I'm told we can't
>>> upgrade the version until the next major release anyway.
>>> root@cass 0 17:32:28 ~ # uname -r
>>> 3.10.0-957.5.1.el7.x86_64
>>>
>>> Cheers,
>>> Phil
>>>
>>> On Thu, 8 Aug 2019 at 17:35, Jon Haddad <jo...@jonhaddad.com> wrote:
>>>
>>> Any chance you're using NVMe with an older Linux kernel?  I've seen a
>>> *lot* filesystem errors from using older CentOS versions.  You'll want to
>>> be using a version > 4.15.
>>>
>>> On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <
>>> philipoconduin@gmail.com> wrote:
>>>
>>> *@Jeff *- If it was hardware that would explain it all, but do you
>>> think it's possible to have every server in the cluster with a hardware
>>> issue?
>>> The data is sensitive and the customer would lose their mind if I sent
>>> it off-site which is a pity cause I could really do with the help.
>>> The corruption is occurring irregularly on every server and instance and
>>> column family in the cluster.  Out of 72 instances, we are getting maybe 10
>>> corrupt files per day.
>>> We are using vnodes (256) and it is happening in both DC's
>>>
>>> *@Asad *- internode compression is set to ALL on every server.  I have
>>> checked the packets for the private interconnect and I can't see any
>>> dropped packets, there are dropped packets for other interfaces, but not
>>> for the private ones, I will get the network team to double-check this.
>>> The corruption is only on the application schema, we are not getting
>>> corruption on any system or cass keyspaces.  Corruption is happening in
>>> both DC's.  We are getting corruption for the 1 application schema we have
>>> across all tables in the keyspace, it's not limited to one table.
>>> Im not sure why the app team decided to not use default compression, I
>>> must ask them.
>>>
>>>
>>>
>>> I have been checking the /var/log/messages today going back a few weeks
>>> and can see a serious amount of broken pipe errors across all servers and
>>> instances.
>>> Here is a snippet from one server but most pipe errors are similar:
>>>
>>> Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
>>> Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072
>>> ops, 0%/0% of on/off-heap limit)
>>> Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during
>>> write!
>>> Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:22  cassandra: at
>>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>>> ~[libthrift-0.9.2.jar:0.9.2]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:22  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during
>>> write!
>>> Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>>> ~[na:1.8.0_172]
>>> Jul  9 03:00:30  cassandra: at
>>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>>> ~[libthrift-0.9.2.jar:0.9.2]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>>> ~[thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:30  cassandra: at
>>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>>> [thrift-server-0.3.7.jar:na]
>>> Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>> Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>>
>>>
>>>
>>> On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:
>>>
>>> Did you check if packets are NOT being dropped for network interfaces
>>> Cassandra instances are consuming (ifconfig –a) internode compression is
>>> set for all endpoint – may be network is playing any role here?
>>>
>>> is this corruption limited so certain keyspace/table | DCs or is that
>>> wide spread – the log snippet you shared it looked like only specific
>>> keyspace/table is affected – is that correct?
>>>
>>> When you remove corrupted sstable of a certain table, I guess you
>>> verifies all nodes for corrupted sstables for same table (may be with with
>>> nodetool scrub tool) so to limit spread of corruptions – right?
>>>
>>> Just curious to know – you’re not using lz4/default compressor for all
>>> tables there must be some reason for it.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
>>> *Sent:* Thursday, August 08, 2019 6:20 AM
>>> *To:* user@cassandra.apache.org
>>> *Subject:* Re: Datafile Corruption
>>>
>>>
>>>
>>> Hi All,
>>>
>>> Thank you so much for the replies.
>>>
>>> Currently, I have the following list that can potentially cause some
>>> sort of corruption in a Cassandra cluster.
>>>
>>>    - Sudden Power cut  -  *We have had no power cuts in the datacenters*
>>>    - Network Issues - *no network issues from what I can tell*
>>>    - Disk full - * I don't think this is an issue for us, see disks
>>>    below.*
>>>    - An issue in Casandra version like Cassandra-13752 -* couldn't find
>>>    any Jira issues similar to ours.*
>>>    - Bit Flips -* we have compression enabled so I don't think this
>>>    should be an issue.*
>>>    - Repair during upgrade has caused corruption too -* we have not
>>>    upgraded*
>>>    - Dropping and adding columns with the same name but a different
>>>    type - *I will need to ask the apps team how they are using the
>>>    database.*
>>>
>>>
>>>
>>> Ok, let me try and explain the issue we are having, I am under a lot of
>>> pressure from above to get this fixed and I can't figure it out.
>>>
>>> This is a PRE-PROD environment.
>>>
>>>    - 2 datacenters.
>>>    - 9 physical servers in each datacenter
>>>    - 4 Cassandra instances on each server
>>>    - 72 Cassandra instances across the 2 data centres, 36 in site A, 36
>>>    in site B.
>>>
>>>
>>> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
>>> datacenter each running with its own Cassandra back end in a cluster
>>> together.
>>>
>>> OS Details [Red Hat Linux]
>>> cass_a@x 0 10:53:01 ~ $ uname -a
>>> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
>>> x86_64 x86_64 x86_64 GNU/Linux
>>>
>>> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
>>> NAME="Red Hat Enterprise Linux Server"
>>> VERSION="7.6 (Maipo)"
>>> ID="rhel"
>>>
>>> Storage Layout
>>> cass_a@xx 0 10:46:28 ~ $ df -h
>>> Filesystem                         Size  Used Avail Use% Mounted on
>>> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
>>> devtmpfs                            63G     0   63G   0% /dev
>>> tmpfs                               63G     0   63G   0% /dev/shm
>>> tmpfs                               63G  4.1G   59G   7% /run
>>> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>>> >> 4 cassandra instances
>>> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
>>> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
>>> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
>>> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>>>  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>> Cassandra load is about 200GB and the rest of the space is snapshots
>>>
>>> CPU
>>> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
>>> CPU(s):                64
>>> Thread(s) per core:    2
>>> Core(s) per socket:    16
>>> Socket(s):             2
>>>
>>> *Description of problem:*
>>> During repair of the cluster, we are seeing multiple corruptions in the
>>> log files on a lot of instances.  There seems to be no pattern to the
>>> corruption.  It seems that the repair job is finding all the corrupted
>>> files for us.  The repair will hang on the node where the corrupted file is
>>> found.  To fix this we remove/rename the datafile and bounce the Cassandra
>>> instance.  Our hardware/OS team have stated there is no problem on their
>>> side.  I do not believe it the repair causing the corruption.
>>>
>>> We have maintenance scripts that run every night running compactions and
>>> creating snapshots, I decided to turn these off, fix any corruptions we
>>> currently had and ran major compaction on all nodes, once this was done we
>>> had a "clean" cluster and we left the cluster for a few days.  After the
>>> process we noticed one corruption in the cluster, this datafile was created
>>> after I turned off the maintenance scripts so my theory of the scripts
>>> causing the issue was wrong.  We then kicked off another repair and started
>>> to find more corrupt files created after the maintenance script was turned
>>> off.
>>>
>>>
>>> So let me give you an example of a corrupted file and maybe someone
>>> might be able to work through it with me?
>>>
>>> When this corrupted file was reported in the log it looks like it was
>>> the repair that found it.
>>>
>>> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
>>> --until "2019-08-07 22:45:00"
>>>
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
>>> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
>>> ops, 0%/0% of on/off-heap limit)
>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a
>>> merkle tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
>>> KeyspaceMetadata/x, (-1476350953672479093,-1474461
>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
>>> Thread[ValidationExecutor:825,1,main]
>>> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
>>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>>> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.repair.Validator.add(Validator.java:150)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>> ~[na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>> [na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>>> /data/ssd2/data/KeyspaceMetadata/x-x/l
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
>>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>>> org.apache.cassandra.io.compress.CorruptBlockException:
>>> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair,
>>> will not do anticompaction
>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
>>> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
>>> operator request
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
>>> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
>>> state jump to shutdown
>>>
>>>
>>> So I went to the file system to see when this corrupt file was created
>>> and it was created on July 30th at 15.55
>>>
>>> root@x 0 01:14:03 ~ # ls -l
>>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30
>>> 15:55 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>>
>>>
>>>
>>> So I checked /var/log/messages for errors around that time
>>> The only thing that stands out to me is the message "Cannot perform a
>>> full major compaction as repaired and unrepaired sstables cannot be
>>> compacted together", I'm not sure if this would be an issue though and
>>> cause corruption.
>>>
>>> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
>>> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
>>> Jul 30 15:55:06 x audispd: node=x. type=USER_START
>>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
>>> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
>>> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
>>> ses=4294967295 msg='cwd="/"
>>> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
>>> terminal=? res=success'
>>> Jul 30 15:55:14 x tag_audit_log: type=USER_START
>>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>>> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes,
>>> 1 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
>>> compaction as repaired and unrepaired sstables cannot be compacted
>>> together. These two set of sstables will be compacted separately.
>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>>> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes,
>>> 57 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
>>> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes,
>>> 1 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>>> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
>>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>>> terminal=/dev/pts/0 res=success'
>>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>>> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
>>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>>> terminal=/dev/pts/1 res=success'
>>>
>>> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
>>> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
>>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>>> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>>> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
>>> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
>>> res=success'
>>> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>>> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes,
>>> 1 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
>>> compaction as repaired and unrepaired sstables cannot be compacted
>>> together. These two set of sstables will be compacted separately.
>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>>> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes,
>>> 16 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
>>> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
>>> ses=4294967295 msg='cwd="/"
>>> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
>>> terminal=? res=success'
>>>
>>>
>>>
>>> We have checked a number of other things like NTP setting etc but
>>> nothing is telling us what could cause so many corruptions across the
>>> entire cluster.
>>> Things were healthy with this cluster for months, the only thing I can
>>> think is that we started loading data from a load of 20GB per instance up
>>> to 200GB where it sits now, maybe this just highlighted the issue.
>>>
>>>
>>>
>>> Compaction and Compression on Keyspace CL's [mixture]
>>> All CF's are using compression.
>>>
>>> AND compaction = {'min_threshold': '4', 'class':
>>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>>> 'max_threshold': '32'}
>>> AND compression = {'sstable_compression':
>>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>>
>>> AND compaction = {'min_threshold': '4', 'class':
>>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>>> 'max_threshold': '32'}
>>> AND compression = {'sstable_compression':
>>> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>>>
>>> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
>>> *LeveledCompactionStrategy*'}
>>> AND compression = {'sstable_compression':
>>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>>
>>> --We are also using internode network compression:
>>> internode_compression: all
>>>
>>>
>>>
>>> Does anyone have any idea what I should check next?
>>> Our next theory is that there may be an issue with Checksum, but I'm not
>>> sure where to go with this.
>>>
>>>
>>>
>>> Any help would be very much appreciated before I lose the last bit of
>>> hair I have on my head.
>>>
>>>
>>>
>>> Kind Regards,
>>>
>>> Phil
>>>
>>>
>>>
>>> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>>>
>>> Repair during upgrade have caused corruption too.
>>>
>>>
>>>
>>> Also, dropping and adding columns with same name but different type
>>>
>>>
>>>
>>> Regards,
>>>
>>> Nitan
>>>
>>> Cell: 510 449 9629
>>>
>>>
>>> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>>>
>>> Is compression enabled?
>>>
>>>
>>>
>>> If not, bit flips on disk can corrupt data files and reads + repair may
>>> send that corruption to other hosts in the cluster
>>>
>>>
>>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
>>> wrote:
>>>
>>> Hi All,
>>>
>>>
>>>
>>> I am currently experiencing multiple datafile corruptions across most
>>> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
>>> starting to think it might be a bug, we're using Cassandra 2.2.13.
>>>
>>>
>>>
>>> Without going into detail about the issue I just want to confirm
>>> something.
>>>
>>>
>>>
>>> Can someone share with me a list of scenarios that would cause
>>> corruption?
>>>
>>>
>>>
>>> 1. OS failure
>>>
>>> 2. Cassandra disturbed during the writing
>>>
>>>
>>>
>>> etc etc.
>>>
>>>
>>>
>>> I need to investigate each scenario and don't want to leave any out.
>>>
>>>
>>>
>>> --
>>>
>>> Regards,
>>>
>>> Phil
>>>
>>>
>>>
>>>
>>> --
>>>
>>> Regards,
>>>
>>> Phil
>>>
>>>
>>>
>>> --
>>> Regards,
>>> Phil
>>>
>>>
>>>
>>> --
>>> Regards,
>>> Phil
>>>
>>
>
> --
> Regards,
> Phil
>


-- 

regards,
Laxmikant Upadhyay

Re: Datafile Corruption

Posted by Philip Ó Condúin <ph...@gmail.com>.
Just to follow up on this issue as others may see it in the future, we
cracked it!

Ou datafile corruption issues were a problem with the OS wrongly taking one
block belonging to a C* data file thinking it was no longer used and
treating it as a free block that would later be used.

For example:
C* deletes file after compaction, OS collects all blocks which are free now
and sends TRIM command to SSD, but SSD from time to time picks the wrong
block, not the one reported by OS - does the trim - causing zeroized blocks
to be seen in the datafile and later use it for different file.
So the symptom is - we suddenly see 4096 zeroes in the datafile- it means
SSD just trimmed the block, after some time we can see some data written to
those blocks - it means the block is used by other file and therefore gives
us a corrupt file.

We turned off the scheduled TRIM function on the OS and we are no longer
getting corruptions.

This was very difficult to pinpoint.

On Thu, 15 Aug 2019 at 00:09, Patrick McFadin <pm...@gmail.com> wrote:

> If you hadn't mentioned the fact you are using physical disk I would have
> guessed you were using virtual disks on a SAN. I've seen this sort of thing
> happen a lot there. Are there any virtual layers between the cassandra
> process and the hardware? Just a reminder, fsync can be a liar and the
> virtual layer can mock the response back to user land while the actual bits
> can be dropped before hitting the disk.
>
> If not, you should be looking hard at your disk options. fstab,
> schedulers, etc. In that case, you need this:
> https://tobert.github.io/pages/als-cassandra-21-tuning-guide.html
>
>
> Patrick
>
> On Wed, Aug 14, 2019 at 2:03 PM Forkalsrud, Erik <ef...@cj.com>
> wrote:
>
>> The dmesg command will usually show information about hardware errors.
>>
>> An example from a spinning disk:
>>     sd 0:0:10:0: [sdi] Unhandled sense code
>>     sd 0:0:10:0: [sdi] Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
>>     sd 0:0:10:0: [sdi] Sense Key : Medium Error [current]
>>     Info fld=0x6fc72
>>     sd 0:0:10:0: [sdi] Add. Sense: Unrecovered read error
>>     sd 0:0:10:0: [sdi] CDB: Read(10): 28 00 00 06 fc 70 00 00 08 00
>>
>>
>> Also, you can read the file like
>> "cat  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db >
>> /dev/null"
>> If you get an error message, it's probably a hardware issue.
>>
>> - Erik -
>>
>> ------------------------------
>> *From:* Philip Ó Condúin <ph...@gmail.com>
>> *Sent:* Thursday, August 8, 2019 09:58
>> *To:* user@cassandra.apache.org <us...@cassandra.apache.org>
>> *Subject:* Re: Datafile Corruption
>>
>> Hi Jon,
>>
>> Good question, I'm not sure if we're using NVMe, I don't see /dev/nvme
>> but we could still be using it.
>> We using *Cisco UCS C220 M4 SFF* so I'm just going to check the spec.
>>
>> Our Kernal is the following, we're using REDHAT so I'm told we can't
>> upgrade the version until the next major release anyway.
>> root@cass 0 17:32:28 ~ # uname -r
>> 3.10.0-957.5.1.el7.x86_64
>>
>> Cheers,
>> Phil
>>
>> On Thu, 8 Aug 2019 at 17:35, Jon Haddad <jo...@jonhaddad.com> wrote:
>>
>> Any chance you're using NVMe with an older Linux kernel?  I've seen a
>> *lot* filesystem errors from using older CentOS versions.  You'll want to
>> be using a version > 4.15.
>>
>> On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <ph...@gmail.com>
>> wrote:
>>
>> *@Jeff *- If it was hardware that would explain it all, but do you think
>> it's possible to have every server in the cluster with a hardware issue?
>> The data is sensitive and the customer would lose their mind if I sent it
>> off-site which is a pity cause I could really do with the help.
>> The corruption is occurring irregularly on every server and instance and
>> column family in the cluster.  Out of 72 instances, we are getting maybe 10
>> corrupt files per day.
>> We are using vnodes (256) and it is happening in both DC's
>>
>> *@Asad *- internode compression is set to ALL on every server.  I have
>> checked the packets for the private interconnect and I can't see any
>> dropped packets, there are dropped packets for other interfaces, but not
>> for the private ones, I will get the network team to double-check this.
>> The corruption is only on the application schema, we are not getting
>> corruption on any system or cass keyspaces.  Corruption is happening in
>> both DC's.  We are getting corruption for the 1 application schema we have
>> across all tables in the keyspace, it's not limited to one table.
>> Im not sure why the app team decided to not use default compression, I
>> must ask them.
>>
>>
>>
>> I have been checking the /var/log/messages today going back a few weeks
>> and can see a serious amount of broken pipe errors across all servers and
>> instances.
>> Here is a snippet from one server but most pipe errors are similar:
>>
>> Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
>> Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072
>> ops, 0%/0% of on/off-heap limit)
>> Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during
>> write!
>> Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>> ~[libthrift-0.9.2.jar:0.9.2]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during
>> write!
>> Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>> ~[libthrift-0.9.2.jar:0.9.2]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>
>>
>>
>> On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:
>>
>> Did you check if packets are NOT being dropped for network interfaces
>> Cassandra instances are consuming (ifconfig –a) internode compression is
>> set for all endpoint – may be network is playing any role here?
>>
>> is this corruption limited so certain keyspace/table | DCs or is that
>> wide spread – the log snippet you shared it looked like only specific
>> keyspace/table is affected – is that correct?
>>
>> When you remove corrupted sstable of a certain table, I guess you
>> verifies all nodes for corrupted sstables for same table (may be with with
>> nodetool scrub tool) so to limit spread of corruptions – right?
>>
>> Just curious to know – you’re not using lz4/default compressor for all
>> tables there must be some reason for it.
>>
>>
>>
>>
>>
>>
>>
>> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
>> *Sent:* Thursday, August 08, 2019 6:20 AM
>> *To:* user@cassandra.apache.org
>> *Subject:* Re: Datafile Corruption
>>
>>
>>
>> Hi All,
>>
>> Thank you so much for the replies.
>>
>> Currently, I have the following list that can potentially cause some sort
>> of corruption in a Cassandra cluster.
>>
>>    - Sudden Power cut  -  *We have had no power cuts in the datacenters*
>>    - Network Issues - *no network issues from what I can tell*
>>    - Disk full - * I don't think this is an issue for us, see disks
>>    below.*
>>    - An issue in Casandra version like Cassandra-13752 -* couldn't find
>>    any Jira issues similar to ours.*
>>    - Bit Flips -* we have compression enabled so I don't think this
>>    should be an issue.*
>>    - Repair during upgrade has caused corruption too -* we have not
>>    upgraded*
>>    - Dropping and adding columns with the same name but a different type
>>    - *I will need to ask the apps team how they are using the database.*
>>
>>
>>
>> Ok, let me try and explain the issue we are having, I am under a lot of
>> pressure from above to get this fixed and I can't figure it out.
>>
>> This is a PRE-PROD environment.
>>
>>    - 2 datacenters.
>>    - 9 physical servers in each datacenter
>>    - 4 Cassandra instances on each server
>>    - 72 Cassandra instances across the 2 data centres, 36 in site A, 36
>>    in site B.
>>
>>
>> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
>> datacenter each running with its own Cassandra back end in a cluster
>> together.
>>
>> OS Details [Red Hat Linux]
>> cass_a@x 0 10:53:01 ~ $ uname -a
>> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
>> x86_64 x86_64 x86_64 GNU/Linux
>>
>> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
>> NAME="Red Hat Enterprise Linux Server"
>> VERSION="7.6 (Maipo)"
>> ID="rhel"
>>
>> Storage Layout
>> cass_a@xx 0 10:46:28 ~ $ df -h
>> Filesystem                         Size  Used Avail Use% Mounted on
>> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
>> devtmpfs                            63G     0   63G   0% /dev
>> tmpfs                               63G     0   63G   0% /dev/shm
>> tmpfs                               63G  4.1G   59G   7% /run
>> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>> >> 4 cassandra instances
>> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
>> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
>> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
>> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>>  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>> Cassandra load is about 200GB and the rest of the space is snapshots
>>
>> CPU
>> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
>> CPU(s):                64
>> Thread(s) per core:    2
>> Core(s) per socket:    16
>> Socket(s):             2
>>
>> *Description of problem:*
>> During repair of the cluster, we are seeing multiple corruptions in the
>> log files on a lot of instances.  There seems to be no pattern to the
>> corruption.  It seems that the repair job is finding all the corrupted
>> files for us.  The repair will hang on the node where the corrupted file is
>> found.  To fix this we remove/rename the datafile and bounce the Cassandra
>> instance.  Our hardware/OS team have stated there is no problem on their
>> side.  I do not believe it the repair causing the corruption.
>>
>> We have maintenance scripts that run every night running compactions and
>> creating snapshots, I decided to turn these off, fix any corruptions we
>> currently had and ran major compaction on all nodes, once this was done we
>> had a "clean" cluster and we left the cluster for a few days.  After the
>> process we noticed one corruption in the cluster, this datafile was created
>> after I turned off the maintenance scripts so my theory of the scripts
>> causing the issue was wrong.  We then kicked off another repair and started
>> to find more corrupt files created after the maintenance script was turned
>> off.
>>
>>
>> So let me give you an example of a corrupted file and maybe someone might
>> be able to work through it with me?
>>
>> When this corrupted file was reported in the log it looks like it was the
>> repair that found it.
>>
>> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
>> --until "2019-08-07 22:45:00"
>>
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
>> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle
>> tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
>> KeyspaceMetadata/x, (-1476350953672479093,-1474461
>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
>> Thread[ValidationExecutor:825,1,main]
>> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.repair.Validator.add(Validator.java:150)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>> ~[na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>> [na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>> /data/ssd2/data/KeyspaceMetadata/x-x/l
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>> org.apache.cassandra.io.compress.CorruptBlockException:
>> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair,
>> will not do anticompaction
>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
>> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
>> operator request
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
>> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
>> state jump to shutdown
>>
>>
>> So I went to the file system to see when this corrupt file was created
>> and it was created on July 30th at 15.55
>>
>> root@x 0 01:14:03 ~ # ls -l
>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30
>> 15:55 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>
>>
>>
>> So I checked /var/log/messages for errors around that time
>> The only thing that stands out to me is the message "Cannot perform a
>> full major compaction as repaired and unrepaired sstables cannot be
>> compacted together", I'm not sure if this would be an issue though and
>> cause corruption.
>>
>> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
>> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
>> Jul 30 15:55:06 x audispd: node=x. type=USER_START
>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
>> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
>> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
>> ses=4294967295 msg='cwd="/"
>> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
>> terminal=? res=success'
>> Jul 30 15:55:14 x tag_audit_log: type=USER_START
>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
>> compaction as repaired and unrepaired sstables cannot be compacted
>> together. These two set of sstables will be compacted separately.
>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes,
>> 57 ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
>> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>> terminal=/dev/pts/0 res=success'
>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>> terminal=/dev/pts/1 res=success'
>>
>> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
>> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
>> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
>> res=success'
>> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
>> compaction as repaired and unrepaired sstables cannot be compacted
>> together. These two set of sstables will be compacted separately.
>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes,
>> 16 ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
>> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
>> ses=4294967295 msg='cwd="/"
>> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
>> terminal=? res=success'
>>
>>
>>
>> We have checked a number of other things like NTP setting etc but nothing
>> is telling us what could cause so many corruptions across the entire
>> cluster.
>> Things were healthy with this cluster for months, the only thing I can
>> think is that we started loading data from a load of 20GB per instance up
>> to 200GB where it sits now, maybe this just highlighted the issue.
>>
>>
>>
>> Compaction and Compression on Keyspace CL's [mixture]
>> All CF's are using compression.
>>
>> AND compaction = {'min_threshold': '4', 'class':
>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>> 'max_threshold': '32'}
>> AND compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>
>> AND compaction = {'min_threshold': '4', 'class':
>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>> 'max_threshold': '32'}
>> AND compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>>
>> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
>> *LeveledCompactionStrategy*'}
>> AND compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>
>> --We are also using internode network compression:
>> internode_compression: all
>>
>>
>>
>> Does anyone have any idea what I should check next?
>> Our next theory is that there may be an issue with Checksum, but I'm not
>> sure where to go with this.
>>
>>
>>
>> Any help would be very much appreciated before I lose the last bit of
>> hair I have on my head.
>>
>>
>>
>> Kind Regards,
>>
>> Phil
>>
>>
>>
>> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>>
>> Repair during upgrade have caused corruption too.
>>
>>
>>
>> Also, dropping and adding columns with same name but different type
>>
>>
>>
>> Regards,
>>
>> Nitan
>>
>> Cell: 510 449 9629
>>
>>
>> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>>
>> Is compression enabled?
>>
>>
>>
>> If not, bit flips on disk can corrupt data files and reads + repair may
>> send that corruption to other hosts in the cluster
>>
>>
>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
>> wrote:
>>
>> Hi All,
>>
>>
>>
>> I am currently experiencing multiple datafile corruptions across most
>> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
>> starting to think it might be a bug, we're using Cassandra 2.2.13.
>>
>>
>>
>> Without going into detail about the issue I just want to confirm
>> something.
>>
>>
>>
>> Can someone share with me a list of scenarios that would cause corruption?
>>
>>
>>
>> 1. OS failure
>>
>> 2. Cassandra disturbed during the writing
>>
>>
>>
>> etc etc.
>>
>>
>>
>> I need to investigate each scenario and don't want to leave any out.
>>
>>
>>
>> --
>>
>> Regards,
>>
>> Phil
>>
>>
>>
>>
>> --
>>
>> Regards,
>>
>> Phil
>>
>>
>>
>> --
>> Regards,
>> Phil
>>
>>
>>
>> --
>> Regards,
>> Phil
>>
>

-- 
Regards,
Phil

Re: Datafile Corruption

Posted by Patrick McFadin <pm...@gmail.com>.
If you hadn't mentioned the fact you are using physical disk I would have
guessed you were using virtual disks on a SAN. I've seen this sort of thing
happen a lot there. Are there any virtual layers between the cassandra
process and the hardware? Just a reminder, fsync can be a liar and the
virtual layer can mock the response back to user land while the actual bits
can be dropped before hitting the disk.

If not, you should be looking hard at your disk options. fstab, schedulers,
etc. In that case, you need this:
https://tobert.github.io/pages/als-cassandra-21-tuning-guide.html


Patrick

On Wed, Aug 14, 2019 at 2:03 PM Forkalsrud, Erik <ef...@cj.com> wrote:

> The dmesg command will usually show information about hardware errors.
>
> An example from a spinning disk:
>     sd 0:0:10:0: [sdi] Unhandled sense code
>     sd 0:0:10:0: [sdi] Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
>     sd 0:0:10:0: [sdi] Sense Key : Medium Error [current]
>     Info fld=0x6fc72
>     sd 0:0:10:0: [sdi] Add. Sense: Unrecovered read error
>     sd 0:0:10:0: [sdi] CDB: Read(10): 28 00 00 06 fc 70 00 00 08 00
>
>
> Also, you can read the file like
> "cat  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db >
> /dev/null"
> If you get an error message, it's probably a hardware issue.
>
> - Erik -
>
> ------------------------------
> *From:* Philip Ó Condúin <ph...@gmail.com>
> *Sent:* Thursday, August 8, 2019 09:58
> *To:* user@cassandra.apache.org <us...@cassandra.apache.org>
> *Subject:* Re: Datafile Corruption
>
> Hi Jon,
>
> Good question, I'm not sure if we're using NVMe, I don't see /dev/nvme but
> we could still be using it.
> We using *Cisco UCS C220 M4 SFF* so I'm just going to check the spec.
>
> Our Kernal is the following, we're using REDHAT so I'm told we can't
> upgrade the version until the next major release anyway.
> root@cass 0 17:32:28 ~ # uname -r
> 3.10.0-957.5.1.el7.x86_64
>
> Cheers,
> Phil
>
> On Thu, 8 Aug 2019 at 17:35, Jon Haddad <jo...@jonhaddad.com> wrote:
>
> Any chance you're using NVMe with an older Linux kernel?  I've seen a
> *lot* filesystem errors from using older CentOS versions.  You'll want to
> be using a version > 4.15.
>
> On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <ph...@gmail.com>
> wrote:
>
> *@Jeff *- If it was hardware that would explain it all, but do you think
> it's possible to have every server in the cluster with a hardware issue?
> The data is sensitive and the customer would lose their mind if I sent it
> off-site which is a pity cause I could really do with the help.
> The corruption is occurring irregularly on every server and instance and
> column family in the cluster.  Out of 72 instances, we are getting maybe 10
> corrupt files per day.
> We are using vnodes (256) and it is happening in both DC's
>
> *@Asad *- internode compression is set to ALL on every server.  I have
> checked the packets for the private interconnect and I can't see any
> dropped packets, there are dropped packets for other interfaces, but not
> for the private ones, I will get the network team to double-check this.
> The corruption is only on the application schema, we are not getting
> corruption on any system or cass keyspaces.  Corruption is happening in
> both DC's.  We are getting corruption for the 1 application schema we have
> across all tables in the keyspace, it's not limited to one table.
> Im not sure why the app team decided to not use default compression, I
> must ask them.
>
>
>
> I have been checking the /var/log/messages today going back a few weeks
> and can see a serious amount of broken pipe errors across all servers and
> instances.
> Here is a snippet from one server but most pipe errors are similar:
>
> Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
> Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072
> ops, 0%/0% of on/off-heap limit)
> Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during write!
> Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
> Jul  9 03:00:22  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native
> Method) ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
> ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
> ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
> ~[libthrift-0.9.2.jar:0.9.2]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.Message.write(Message.java:222)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during write!
> Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
> Jul  9 03:00:30  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native
> Method) ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
> ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
> ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
> ~[libthrift-0.9.2.jar:0.9.2]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.Message.write(Message.java:222)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>
>
>
> On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:
>
> Did you check if packets are NOT being dropped for network interfaces
> Cassandra instances are consuming (ifconfig –a) internode compression is
> set for all endpoint – may be network is playing any role here?
>
> is this corruption limited so certain keyspace/table | DCs or is that wide
> spread – the log snippet you shared it looked like only specific
> keyspace/table is affected – is that correct?
>
> When you remove corrupted sstable of a certain table, I guess you verifies
> all nodes for corrupted sstables for same table (may be with with nodetool
> scrub tool) so to limit spread of corruptions – right?
>
> Just curious to know – you’re not using lz4/default compressor for all
> tables there must be some reason for it.
>
>
>
>
>
>
>
> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
> *Sent:* Thursday, August 08, 2019 6:20 AM
> *To:* user@cassandra.apache.org
> *Subject:* Re: Datafile Corruption
>
>
>
> Hi All,
>
> Thank you so much for the replies.
>
> Currently, I have the following list that can potentially cause some sort
> of corruption in a Cassandra cluster.
>
>    - Sudden Power cut  -  *We have had no power cuts in the datacenters*
>    - Network Issues - *no network issues from what I can tell*
>    - Disk full - * I don't think this is an issue for us, see disks
>    below.*
>    - An issue in Casandra version like Cassandra-13752 -* couldn't find
>    any Jira issues similar to ours.*
>    - Bit Flips -* we have compression enabled so I don't think this
>    should be an issue.*
>    - Repair during upgrade has caused corruption too -* we have not
>    upgraded*
>    - Dropping and adding columns with the same name but a different type
>    - *I will need to ask the apps team how they are using the database.*
>
>
>
> Ok, let me try and explain the issue we are having, I am under a lot of
> pressure from above to get this fixed and I can't figure it out.
>
> This is a PRE-PROD environment.
>
>    - 2 datacenters.
>    - 9 physical servers in each datacenter
>    - 4 Cassandra instances on each server
>    - 72 Cassandra instances across the 2 data centres, 36 in site A, 36
>    in site B.
>
>
> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
> datacenter each running with its own Cassandra back end in a cluster
> together.
>
> OS Details [Red Hat Linux]
> cass_a@x 0 10:53:01 ~ $ uname -a
> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
> x86_64 x86_64 x86_64 GNU/Linux
>
> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
> NAME="Red Hat Enterprise Linux Server"
> VERSION="7.6 (Maipo)"
> ID="rhel"
>
> Storage Layout
> cass_a@xx 0 10:46:28 ~ $ df -h
> Filesystem                         Size  Used Avail Use% Mounted on
> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
> devtmpfs                            63G     0   63G   0% /dev
> tmpfs                               63G     0   63G   0% /dev/shm
> tmpfs                               63G  4.1G   59G   7% /run
> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
> >> 4 cassandra instances
> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
> Cassandra load is about 200GB and the rest of the space is snapshots
>
> CPU
> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
> CPU(s):                64
> Thread(s) per core:    2
> Core(s) per socket:    16
> Socket(s):             2
>
> *Description of problem:*
> During repair of the cluster, we are seeing multiple corruptions in the
> log files on a lot of instances.  There seems to be no pattern to the
> corruption.  It seems that the repair job is finding all the corrupted
> files for us.  The repair will hang on the node where the corrupted file is
> found.  To fix this we remove/rename the datafile and bounce the Cassandra
> instance.  Our hardware/OS team have stated there is no problem on their
> side.  I do not believe it the repair causing the corruption.
>
> We have maintenance scripts that run every night running compactions and
> creating snapshots, I decided to turn these off, fix any corruptions we
> currently had and ran major compaction on all nodes, once this was done we
> had a "clean" cluster and we left the cluster for a few days.  After the
> process we noticed one corruption in the cluster, this datafile was created
> after I turned off the maintenance scripts so my theory of the scripts
> causing the issue was wrong.  We then kicked off another repair and started
> to find more corrupt files created after the maintenance script was turned
> off.
>
>
> So let me give you an example of a corrupted file and maybe someone might
> be able to work through it with me?
>
> When this corrupted file was reported in the log it looks like it was the
> repair that found it.
>
> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
> --until "2019-08-07 22:45:00"
>
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle
> tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
> KeyspaceMetadata/x, (-1476350953672479093,-1474461
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
> Thread[ValidationExecutor:825,1,main]
> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.repair.Validator.add(Validator.java:150)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> ~[na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> [na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at java.lang.Thread.run(Thread.java:748)
> [na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: Caused by:
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
> /data/ssd2/data/KeyspaceMetadata/x-x/l
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
> Aug 07 22:30:33 cassandra[34611]: Caused by:
> org.apache.cassandra.io.compress.CorruptBlockException:
> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair, will
> not do anticompaction
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
> operator request
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
> state jump to shutdown
>
>
> So I went to the file system to see when this corrupt file was created and
> it was created on July 30th at 15.55
>
> root@x 0 01:14:03 ~ # ls -l
> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30
> 15:55 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>
>
>
> So I checked /var/log/messages for errors around that time
> The only thing that stands out to me is the message "Cannot perform a full
> major compaction as repaired and unrepaired sstables cannot be compacted
> together", I'm not sure if this would be an issue though and cause
> corruption.
>
> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
> Jul 30 15:55:06 x audispd: node=x. type=USER_START
> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
> ses=4294967295 msg='cwd="/"
> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
> terminal=? res=success'
> Jul 30 15:55:14 x tag_audit_log: type=USER_START
> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
> compaction as repaired and unrepaired sstables cannot be compacted
> together. These two set of sstables will be compacted separately.
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes, 57
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
> terminal=/dev/pts/0 res=success'
> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
> terminal=/dev/pts/1 res=success'
>
> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
> Jul 30 15:55:57 x audispd: node=x. type=USER_START
> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:57 x audispd: node=x. type=USER_START
> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
> res=success'
> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
> compaction as repaired and unrepaired sstables cannot be compacted
> together. These two set of sstables will be compacted separately.
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes, 16
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
> ses=4294967295 msg='cwd="/"
> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
> terminal=? res=success'
>
>
>
> We have checked a number of other things like NTP setting etc but nothing
> is telling us what could cause so many corruptions across the entire
> cluster.
> Things were healthy with this cluster for months, the only thing I can
> think is that we started loading data from a load of 20GB per instance up
> to 200GB where it sits now, maybe this just highlighted the issue.
>
>
>
> Compaction and Compression on Keyspace CL's [mixture]
> All CF's are using compression.
>
> AND compaction = {'min_threshold': '4', 'class':
> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
> 'max_threshold': '32'}
> AND compression = {'sstable_compression':
> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>
> AND compaction = {'min_threshold': '4', 'class':
> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
> 'max_threshold': '32'}
> AND compression = {'sstable_compression':
> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>
> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
> *LeveledCompactionStrategy*'}
> AND compression = {'sstable_compression':
> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>
> --We are also using internode network compression:
> internode_compression: all
>
>
>
> Does anyone have any idea what I should check next?
> Our next theory is that there may be an issue with Checksum, but I'm not
> sure where to go with this.
>
>
>
> Any help would be very much appreciated before I lose the last bit of hair
> I have on my head.
>
>
>
> Kind Regards,
>
> Phil
>
>
>
> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>
> Repair during upgrade have caused corruption too.
>
>
>
> Also, dropping and adding columns with same name but different type
>
>
>
> Regards,
>
> Nitan
>
> Cell: 510 449 9629
>
>
> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>
> Is compression enabled?
>
>
>
> If not, bit flips on disk can corrupt data files and reads + repair may
> send that corruption to other hosts in the cluster
>
>
> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
> wrote:
>
> Hi All,
>
>
>
> I am currently experiencing multiple datafile corruptions across most
> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
> starting to think it might be a bug, we're using Cassandra 2.2.13.
>
>
>
> Without going into detail about the issue I just want to confirm something.
>
>
>
> Can someone share with me a list of scenarios that would cause corruption?
>
>
>
> 1. OS failure
>
> 2. Cassandra disturbed during the writing
>
>
>
> etc etc.
>
>
>
> I need to investigate each scenario and don't want to leave any out.
>
>
>
> --
>
> Regards,
>
> Phil
>
>
>
>
> --
>
> Regards,
>
> Phil
>
>
>
> --
> Regards,
> Phil
>
>
>
> --
> Regards,
> Phil
>

Re: Datafile Corruption

Posted by "Forkalsrud, Erik" <ef...@cj.com>.
The dmesg command will usually show information about hardware errors.

An example from a spinning disk:
    sd 0:0:10:0: [sdi] Unhandled sense code
    sd 0:0:10:0: [sdi] Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
    sd 0:0:10:0: [sdi] Sense Key : Medium Error [current]
    Info fld=0x6fc72
    sd 0:0:10:0: [sdi] Add. Sense: Unrecovered read error
    sd 0:0:10:0: [sdi] CDB: Read(10): 28 00 00 06 fc 70 00 00 08 00


Also, you can read the file like  "cat  /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db > /dev/null"
If you get an error message, it's probably a hardware issue.

- Erik -

________________________________
From: Philip Ó Condúin <ph...@gmail.com>
Sent: Thursday, August 8, 2019 09:58
To: user@cassandra.apache.org <us...@cassandra.apache.org>
Subject: Re: Datafile Corruption

Hi Jon,

Good question, I'm not sure if we're using NVMe, I don't see /dev/nvme but we could still be using it.
We using Cisco UCS C220 M4 SFF so I'm just going to check the spec.

Our Kernal is the following, we're using REDHAT so I'm told we can't upgrade the version until the next major release anyway.
root@cass 0 17:32:28 ~ # uname -r
3.10.0-957.5.1.el7.x86_64

Cheers,
Phil

On Thu, 8 Aug 2019 at 17:35, Jon Haddad <jo...@jonhaddad.com>> wrote:
Any chance you're using NVMe with an older Linux kernel?  I've seen a *lot* filesystem errors from using older CentOS versions.  You'll want to be using a version > 4.15.

On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <ph...@gmail.com>> wrote:
@Jeff - If it was hardware that would explain it all, but do you think it's possible to have every server in the cluster with a hardware issue?
The data is sensitive and the customer would lose their mind if I sent it off-site which is a pity cause I could really do with the help.
The corruption is occurring irregularly on every server and instance and column family in the cluster.  Out of 72 instances, we are getting maybe 10 corrupt files per day.
We are using vnodes (256) and it is happening in both DC's

@Asad - internode compression is set to ALL on every server.  I have checked the packets for the private interconnect and I can't see any dropped packets, there are dropped packets for other interfaces, but not for the private ones, I will get the network team to double-check this.
The corruption is only on the application schema, we are not getting corruption on any system or cass keyspaces.  Corruption is happening in both DC's.  We are getting corruption for the 1 application schema we have across all tables in the keyspace, it's not limited to one table.
Im not sure why the app team decided to not use default compression, I must ask them.



I have been checking the /var/log/messages today going back a few weeks and can see a serious amount of broken pipe errors across all servers and instances.
Here is a snippet from one server but most pipe errors are similar:

Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072 ops, 0%/0% of on/off-heap limit)
Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during write!
Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
Jul  9 03:00:22  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165) ~[libthrift-0.9.2.jar:0.9.2]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104) ~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112) ~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.Message.write(Message.java:222) ~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during write!
Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
Jul  9 03:00:30  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165) ~[libthrift-0.9.2.jar:0.9.2]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104) ~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112) ~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.Message.write(Message.java:222) ~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383) [thrift-server-0.3.7.jar:na]
Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed



On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com>> wrote:

Did you check if packets are NOT being dropped for network interfaces Cassandra instances are consuming (ifconfig –a) internode compression is set for all endpoint – may be network is playing any role here?

is this corruption limited so certain keyspace/table | DCs or is that wide spread – the log snippet you shared it looked like only specific keyspace/table is affected – is that correct?

When you remove corrupted sstable of a certain table, I guess you verifies all nodes for corrupted sstables for same table (may be with with nodetool scrub tool) so to limit spread of corruptions – right?

Just curious to know – you’re not using lz4/default compressor for all tables there must be some reason for it.







From: Philip Ó Condúin [mailto:philipoconduin@gmail.com<ma...@gmail.com>]
Sent: Thursday, August 08, 2019 6:20 AM
To: user@cassandra.apache.org<ma...@cassandra.apache.org>
Subject: Re: Datafile Corruption



Hi All,

Thank you so much for the replies.

Currently, I have the following list that can potentially cause some sort of corruption in a Cassandra cluster.

  *   Sudden Power cut  -  We have had no power cuts in the datacenters
  *   Network Issues - no network issues from what I can tell
  *   Disk full - I don't think this is an issue for us, see disks below.
  *   An issue in Casandra version like Cassandra-13752 - couldn't find any Jira issues similar to ours.
  *   Bit Flips - we have compression enabled so I don't think this should be an issue.
  *   Repair during upgrade has caused corruption too - we have not upgraded
  *   Dropping and adding columns with the same name but a different type - I will need to ask the apps team how they are using the database.


Ok, let me try and explain the issue we are having, I am under a lot of pressure from above to get this fixed and I can't figure it out.

This is a PRE-PROD environment.

  *   2 datacenters.
  *   9 physical servers in each datacenter
  *   4 Cassandra instances on each server
  *   72 Cassandra instances across the 2 data centres, 36 in site A, 36 in site B.

We also have 2 Reaper Nodes we use for repair.  One reaper node in each datacenter each running with its own Cassandra back end in a cluster together.

OS Details [Red Hat Linux]
cass_a@x 0 10:53:01 ~ $ uname -a
Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018 x86_64 x86_64 x86_64 GNU/Linux

cass_a@x 0 10:57:31 ~ $ cat /etc/*release
NAME="Red Hat Enterprise Linux Server"
VERSION="7.6 (Maipo)"
ID="rhel"

Storage Layout
cass_a@xx 0 10:46:28 ~ $ df -h
Filesystem                         Size  Used Avail Use% Mounted on
/dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
devtmpfs                            63G     0   63G   0% /dev
tmpfs                               63G     0   63G   0% /dev/shm
tmpfs                               63G  4.1G   59G   7% /run
tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>> 4 cassandra instances
/dev/sdd                           1.5T  802G  688G  54% /data/ssd4
/dev/sda                           1.5T  798G  692G  54% /data/ssd1
/dev/sdb                           1.5T  681G  810G  46% /data/ssd2
/dev/sdc                           1.5T  558G  932G  38% /data/ssd3
 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
Cassandra load is about 200GB and the rest of the space is snapshots

CPU
cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
CPU(s):                64
Thread(s) per core:    2
Core(s) per socket:    16
Socket(s):             2

Description of problem:
During repair of the cluster, we are seeing multiple corruptions in the log files on a lot of instances.  There seems to be no pattern to the corruption.  It seems that the repair job is finding all the corrupted files for us.  The repair will hang on the node where the corrupted file is found.  To fix this we remove/rename the datafile and bounce the Cassandra instance.  Our hardware/OS team have stated there is no problem on their side.  I do not believe it the repair causing the corruption.

We have maintenance scripts that run every night running compactions and creating snapshots, I decided to turn these off, fix any corruptions we currently had and ran major compaction on all nodes, once this was done we had a "clean" cluster and we left the cluster for a few days.  After the process we noticed one corruption in the cluster, this datafile was created after I turned off the maintenance scripts so my theory of the scripts causing the issue was wrong.  We then kicked off another repair and started to find more corrupt files created after the maintenance script was turned off.


So let me give you an example of a corrupted file and maybe someone might be able to work through it with me?

When this corrupted file was reported in the log it looks like it was the repair that found it.

$ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00" --until "2019-08-07 22:45:00"

Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing Memtable-compactions_in_progress@830377457(0.008KiB<mailto:Memtable-compactions_in_progress@830377457(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on KeyspaceMetadata/x, (-1476350953672479093,-1474461
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread Thread[ValidationExecutor:825,1,main]
Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.Iterators$7.computeNext(Iterators.java:645) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.repair.Validator.rowHash(Validator.java:201) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.repair.Validator.add(Validator.java:150) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /data/ssd2/data/KeyspaceMetadata/x-x/l
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
Aug 07 22:30:33 cassandra[34611]: Caused by: org.apache.cassandra.io.compress.CorruptBlockException: (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair, will not do anticompaction
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by operator request
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37<https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=> state jump to shutdown


So I went to the file system to see when this corrupt file was created and it was created on July 30th at 15.55

root@x 0 01:14:03 ~ # ls -l /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
-rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db



So I checked /var/log/messages for errors around that time
The only thing that stands out to me is the message "Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together", I'm not sure if this would be an issue though and cause corruption.

Jul 30 15:55:06 x systemd: Created slice User Slice of root.
Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
Jul 30 15:55:06 x audispd: node=x. type=USER_START msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
Jul 30 15:55:14 x tag_audit_log: type=USER_CMD msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295 ses=4294967295 msg='cwd="/" cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473 terminal=? res=success'
Jul 30 15:55:14 x tag_audit_log: type=USER_START msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing Memtable-compactions_in_progress@1462227999(0.008KiB<mailto:Memtable-compactions_in_progress@1462227999(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing Memtable-compactions_in_progress@1198535528(1.002KiB<mailto:Memtable-compactions_in_progress@1198535528(1.002KiB> serialized bytes, 57 ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing Memtable-compactions_in_progress@2039409834(0.008KiB<mailto:Memtable-compactions_in_progress@2039409834(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747 msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=? terminal=/dev/pts/0 res=success'
Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749 msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=? terminal=/dev/pts/1 res=success'

Jul 30 15:55:57 x systemd: Created slice User Slice of root.
Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
Jul 30 15:55:57 x audispd: node=x. type=USER_START msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:57 x audispd: node=x. type=USER_START msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing Memtable-compactions_in_progress@1532791194(0.008KiB<mailto:Memtable-compactions_in_progress@1532791194(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing Memtable-compactions_in_progress@1455399453(0.281KiB<mailto:Memtable-compactions_in_progress@1455399453(0.281KiB> serialized bytes, 16 ops, 0%/0% of on/off-heap limit)
Jul 30 15:56:04 x tag_audit_log: type=USER_CMD msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295 ses=4294967295 msg='cwd="/" cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473 terminal=? res=success'



We have checked a number of other things like NTP setting etc but nothing is telling us what could cause so many corruptions across the entire cluster.
Things were healthy with this cluster for months, the only thing I can think is that we started loading data from a load of 20GB per instance up to 200GB where it sits now, maybe this just highlighted the issue.



Compaction and Compression on Keyspace CL's [mixture]
All CF's are using compression.

AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}

AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.LZ4Compressor'}

AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}

--We are also using internode network compression:
internode_compression: all



Does anyone have any idea what I should check next?
Our next theory is that there may be an issue with Checksum, but I'm not sure where to go with this.



Any help would be very much appreciated before I lose the last bit of hair I have on my head.



Kind Regards,

Phil



On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com>> wrote:

Repair during upgrade have caused corruption too.



Also, dropping and adding columns with same name but different type



Regards,

Nitan

Cell: 510 449 9629<tel:510%20449%209629>

On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com>> wrote:

Is compression enabled?



If not, bit flips on disk can corrupt data files and reads + repair may send that corruption to other hosts in the cluster

On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>> wrote:

Hi All,



I am currently experiencing multiple datafile corruptions across most nodes in my cluster, there seems to be no pattern to the corruption.  I'm starting to think it might be a bug, we're using Cassandra 2.2.13.



Without going into detail about the issue I just want to confirm something.



Can someone share with me a list of scenarios that would cause corruption?



1. OS failure

2. Cassandra disturbed during the writing



etc etc.



I need to investigate each scenario and don't want to leave any out.



--

Regards,

Phil




--

Regards,

Phil


--
Regards,
Phil


--
Regards,
Phil

Re: Datafile Corruption

Posted by Philip Ó Condúin <ph...@gmail.com>.
Hi Jon,

Good question, I'm not sure if we're using NVMe, I don't see /dev/nvme but
we could still be using it.
We using *Cisco UCS C220 M4 SFF* so I'm just going to check the spec.

Our Kernal is the following, we're using REDHAT so I'm told we can't
upgrade the version until the next major release anyway.
root@cass 0 17:32:28 ~ # uname -r
3.10.0-957.5.1.el7.x86_64

Cheers,
Phil

On Thu, 8 Aug 2019 at 17:35, Jon Haddad <jo...@jonhaddad.com> wrote:

> Any chance you're using NVMe with an older Linux kernel?  I've seen a
> *lot* filesystem errors from using older CentOS versions.  You'll want to
> be using a version > 4.15.
>
> On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <ph...@gmail.com>
> wrote:
>
>> *@Jeff *- If it was hardware that would explain it all, but do you think
>> it's possible to have every server in the cluster with a hardware issue?
>> The data is sensitive and the customer would lose their mind if I sent it
>> off-site which is a pity cause I could really do with the help.
>> The corruption is occurring irregularly on every server and instance and
>> column family in the cluster.  Out of 72 instances, we are getting maybe 10
>> corrupt files per day.
>> We are using vnodes (256) and it is happening in both DC's
>>
>> *@Asad *- internode compression is set to ALL on every server.  I have
>> checked the packets for the private interconnect and I can't see any
>> dropped packets, there are dropped packets for other interfaces, but not
>> for the private ones, I will get the network team to double-check this.
>> The corruption is only on the application schema, we are not getting
>> corruption on any system or cass keyspaces.  Corruption is happening in
>> both DC's.  We are getting corruption for the 1 application schema we have
>> across all tables in the keyspace, it's not limited to one table.
>> Im not sure why the app team decided to not use default compression, I
>> must ask them.
>>
>>
>>
>> I have been checking the /var/log/messages today going back a few weeks
>> and can see a serious amount of broken pipe errors across all servers and
>> instances.
>> Here is a snippet from one server but most pipe errors are similar:
>>
>> Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
>> Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072
>> ops, 0%/0% of on/off-heap limit)
>> Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during
>> write!
>> Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:22  cassandra: at
>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>> ~[libthrift-0.9.2.jar:0.9.2]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:22  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during
>> write!
>> Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
>> ~[na:1.8.0_172]
>> Jul  9 03:00:30  cassandra: at
>> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
>> ~[libthrift-0.9.2.jar:0.9.2]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.Message.write(Message.java:222)
>> ~[thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:30  cassandra: at
>> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
>> [thrift-server-0.3.7.jar:na]
>> Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>> Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>>
>>
>>
>> On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:
>>
>>> Did you check if packets are NOT being dropped for network interfaces
>>> Cassandra instances are consuming (ifconfig –a) internode compression is
>>> set for all endpoint – may be network is playing any role here?
>>>
>>> is this corruption limited so certain keyspace/table | DCs or is that
>>> wide spread – the log snippet you shared it looked like only specific
>>> keyspace/table is affected – is that correct?
>>>
>>> When you remove corrupted sstable of a certain table, I guess you
>>> verifies all nodes for corrupted sstables for same table (may be with with
>>> nodetool scrub tool) so to limit spread of corruptions – right?
>>>
>>> Just curious to know – you’re not using lz4/default compressor for all
>>> tables there must be some reason for it.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
>>> *Sent:* Thursday, August 08, 2019 6:20 AM
>>> *To:* user@cassandra.apache.org
>>> *Subject:* Re: Datafile Corruption
>>>
>>>
>>>
>>> Hi All,
>>>
>>> Thank you so much for the replies.
>>>
>>> Currently, I have the following list that can potentially cause some
>>> sort of corruption in a Cassandra cluster.
>>>
>>>    - Sudden Power cut  -  *We have had no power cuts in the datacenters*
>>>    - Network Issues - *no network issues from what I can tell*
>>>    - Disk full - *I don't think this is an issue for us, see disks
>>>    below.*
>>>    - An issue in Casandra version like Cassandra-13752 -* couldn't find
>>>    any Jira issues similar to ours.*
>>>    - Bit Flips -* we have compression enabled so I don't think this
>>>    should be an issue.*
>>>    - Repair during upgrade has caused corruption too -* we have not
>>>    upgraded*
>>>    - Dropping and adding columns with the same name but a different
>>>    type - *I will need to ask the apps team how they are using the
>>>    database.*
>>>
>>>
>>>
>>> Ok, let me try and explain the issue we are having, I am under a lot of
>>> pressure from above to get this fixed and I can't figure it out.
>>>
>>> This is a PRE-PROD environment.
>>>
>>>    - 2 datacenters.
>>>    - 9 physical servers in each datacenter
>>>    - 4 Cassandra instances on each server
>>>    - 72 Cassandra instances across the 2 data centres, 36 in site A, 36
>>>    in site B.
>>>
>>>
>>> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
>>> datacenter each running with its own Cassandra back end in a cluster
>>> together.
>>>
>>> OS Details [Red Hat Linux]
>>> cass_a@x 0 10:53:01 ~ $ uname -a
>>> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
>>> x86_64 x86_64 x86_64 GNU/Linux
>>>
>>> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
>>> NAME="Red Hat Enterprise Linux Server"
>>> VERSION="7.6 (Maipo)"
>>> ID="rhel"
>>>
>>> Storage Layout
>>> cass_a@xx 0 10:46:28 ~ $ df -h
>>> Filesystem                         Size  Used Avail Use% Mounted on
>>> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
>>> devtmpfs                            63G     0   63G   0% /dev
>>> tmpfs                               63G     0   63G   0% /dev/shm
>>> tmpfs                               63G  4.1G   59G   7% /run
>>> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>>> >> 4 cassandra instances
>>> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
>>> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
>>> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
>>> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>>>
>>> Cassandra load is about 200GB and the rest of the space is snapshots
>>>
>>> CPU
>>> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
>>> CPU(s):                64
>>> Thread(s) per core:    2
>>> Core(s) per socket:    16
>>> Socket(s):             2
>>>
>>> *Description of problem:*
>>> During repair of the cluster, we are seeing multiple corruptions in the
>>> log files on a lot of instances.  There seems to be no pattern to the
>>> corruption.  It seems that the repair job is finding all the corrupted
>>> files for us.  The repair will hang on the node where the corrupted file is
>>> found.  To fix this we remove/rename the datafile and bounce the Cassandra
>>> instance.  Our hardware/OS team have stated there is no problem on their
>>> side.  I do not believe it the repair causing the corruption.
>>>
>>> We have maintenance scripts that run every night running compactions and
>>> creating snapshots, I decided to turn these off, fix any corruptions we
>>> currently had and ran major compaction on all nodes, once this was done we
>>> had a "clean" cluster and we left the cluster for a few days.  After the
>>> process we noticed one corruption in the cluster, this datafile was created
>>> after I turned off the maintenance scripts so my theory of the scripts
>>> causing the issue was wrong.  We then kicked off another repair and started
>>> to find more corrupt files created after the maintenance script was turned
>>> off.
>>>
>>>
>>> So let me give you an example of a corrupted file and maybe someone
>>> might be able to work through it with me?
>>>
>>> When this corrupted file was reported in the log it looks like it was
>>> the repair that found it.
>>>
>>> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
>>> --until "2019-08-07 22:45:00"
>>>
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
>>> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
>>> ops, 0%/0% of on/off-heap limit)
>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a
>>> merkle tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
>>> KeyspaceMetadata/x, (-1476350953672479093,-1474461
>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
>>> Thread[ValidationExecutor:825,1,main]
>>> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
>>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>>> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>> ~[guava-16.0.jar:na]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.repair.Validator.add(Validator.java:150)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>> ~[na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>> [na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
>>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>>> /data/ssd2/data/KeyspaceMetadata/x-x/l
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
>>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>>> org.apache.cassandra.io.compress.CorruptBlockException:
>>> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
>>> Aug 07 22:30:33 cassandra[34611]: at
>>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
>>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>>> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair,
>>> will not do anticompaction
>>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
>>> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
>>> operator request
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
>>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
>>> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
>>> state jump to shutdown
>>>
>>>
>>> So I went to the file system to see when this corrupt file was created
>>> and it was created on July 30th at 15.55
>>>
>>> root@x 0 01:14:03 ~ # ls -l
>>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55
>>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>>
>>>
>>>
>>> So I checked /var/log/messages for errors around that time
>>> The only thing that stands out to me is the message "Cannot perform a
>>> full major compaction as repaired and unrepaired sstables cannot be
>>> compacted together", I'm not sure if this would be an issue though and
>>> cause corruption.
>>>
>>> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
>>> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
>>> Jul 30 15:55:06 x audispd: node=x. type=USER_START
>>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
>>> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
>>> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
>>> ses=4294967295 msg='cwd="/"
>>> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
>>> terminal=? res=success'
>>> Jul 30 15:55:14 x tag_audit_log: type=USER_START
>>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>>> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes,
>>> 1 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
>>> compaction as repaired and unrepaired sstables cannot be compacted
>>> together. These two set of sstables will be compacted separately.
>>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>>> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes,
>>> 57 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
>>> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes,
>>> 1 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>>> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
>>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>>> terminal=/dev/pts/0 res=success'
>>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>>> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
>>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>>> terminal=/dev/pts/1 res=success'
>>>
>>> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
>>> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
>>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>>> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>>> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
>>> ses=4294967295 msg='op=PAM:session_open
>>> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
>>> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
>>> res=success'
>>> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>>> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes,
>>> 1 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
>>> compaction as repaired and unrepaired sstables cannot be compacted
>>> together. These two set of sstables will be compacted separately.
>>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>>> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes,
>>> 16 ops, 0%/0% of on/off-heap limit)
>>> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
>>> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
>>> ses=4294967295 msg='cwd="/"
>>> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
>>> terminal=? res=success'
>>>
>>>
>>>
>>> We have checked a number of other things like NTP setting etc but
>>> nothing is telling us what could cause so many corruptions across the
>>> entire cluster.
>>> Things were healthy with this cluster for months, the only thing I can
>>> think is that we started loading data from a load of 20GB per instance up
>>> to 200GB where it sits now, maybe this just highlighted the issue.
>>>
>>>
>>>
>>> Compaction and Compression on Keyspace CL's [mixture]
>>> All CF's are using compression.
>>>
>>> AND compaction = {'min_threshold': '4', 'class':
>>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>>> 'max_threshold': '32'}
>>> AND compression = {'sstable_compression':
>>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>>
>>> AND compaction = {'min_threshold': '4', 'class':
>>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>>> 'max_threshold': '32'}
>>> AND compression = {'sstable_compression':
>>> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>>>
>>> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
>>> *LeveledCompactionStrategy*'}
>>> AND compression = {'sstable_compression':
>>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>>
>>> --We are also using internode network compression:
>>> internode_compression: all
>>>
>>>
>>>
>>> Does anyone have any idea what I should check next?
>>> Our next theory is that there may be an issue with Checksum, but I'm not
>>> sure where to go with this.
>>>
>>>
>>>
>>> Any help would be very much appreciated before I lose the last bit of
>>> hair I have on my head.
>>>
>>>
>>>
>>> Kind Regards,
>>>
>>> Phil
>>>
>>>
>>>
>>> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>>>
>>> Repair during upgrade have caused corruption too.
>>>
>>>
>>>
>>> Also, dropping and adding columns with same name but different type
>>>
>>>
>>>
>>> Regards,
>>>
>>> Nitan
>>>
>>> Cell: 510 449 9629
>>>
>>>
>>> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>>>
>>> Is compression enabled?
>>>
>>>
>>>
>>> If not, bit flips on disk can corrupt data files and reads + repair may
>>> send that corruption to other hosts in the cluster
>>>
>>>
>>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
>>> wrote:
>>>
>>> Hi All,
>>>
>>>
>>>
>>> I am currently experiencing multiple datafile corruptions across most
>>> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
>>> starting to think it might be a bug, we're using Cassandra 2.2.13.
>>>
>>>
>>>
>>> Without going into detail about the issue I just want to confirm
>>> something.
>>>
>>>
>>>
>>> Can someone share with me a list of scenarios that would cause
>>> corruption?
>>>
>>>
>>>
>>> 1. OS failure
>>>
>>> 2. Cassandra disturbed during the writing
>>>
>>>
>>>
>>> etc etc.
>>>
>>>
>>>
>>> I need to investigate each scenario and don't want to leave any out.
>>>
>>>
>>>
>>> --
>>>
>>> Regards,
>>>
>>> Phil
>>>
>>>
>>>
>>>
>>> --
>>>
>>> Regards,
>>>
>>> Phil
>>>
>>
>>
>> --
>> Regards,
>> Phil
>>
>

-- 
Regards,
Phil

Re: Datafile Corruption

Posted by Jon Haddad <jo...@jonhaddad.com>.
Any chance you're using NVMe with an older Linux kernel?  I've seen a *lot*
filesystem errors from using older CentOS versions.  You'll want to be
using a version > 4.15.

On Thu, Aug 8, 2019 at 9:31 AM Philip Ó Condúin <ph...@gmail.com>
wrote:

> *@Jeff *- If it was hardware that would explain it all, but do you think
> it's possible to have every server in the cluster with a hardware issue?
> The data is sensitive and the customer would lose their mind if I sent it
> off-site which is a pity cause I could really do with the help.
> The corruption is occurring irregularly on every server and instance and
> column family in the cluster.  Out of 72 instances, we are getting maybe 10
> corrupt files per day.
> We are using vnodes (256) and it is happening in both DC's
>
> *@Asad *- internode compression is set to ALL on every server.  I have
> checked the packets for the private interconnect and I can't see any
> dropped packets, there are dropped packets for other interfaces, but not
> for the private ones, I will get the network team to double-check this.
> The corruption is only on the application schema, we are not getting
> corruption on any system or cass keyspaces.  Corruption is happening in
> both DC's.  We are getting corruption for the 1 application schema we have
> across all tables in the keyspace, it's not limited to one table.
> Im not sure why the app team decided to not use default compression, I
> must ask them.
>
>
>
> I have been checking the /var/log/messages today going back a few weeks
> and can see a serious amount of broken pipe errors across all servers and
> instances.
> Here is a snippet from one server but most pipe errors are similar:
>
> Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
> Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072
> ops, 0%/0% of on/off-heap limit)
> Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during write!
> Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
> Jul  9 03:00:22  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native
> Method) ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
> ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
> ~[na:1.8.0_172]
> Jul  9 03:00:22  cassandra: at
> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
> ~[libthrift-0.9.2.jar:0.9.2]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.Message.write(Message.java:222)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:22  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during write!
> Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
> Jul  9 03:00:30  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native
> Method) ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
> ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
> ~[na:1.8.0_172]
> Jul  9 03:00:30  cassandra: at
> org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
> ~[libthrift-0.9.2.jar:0.9.2]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.Message.write(Message.java:222)
> ~[thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:30  cassandra: at
> com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
> [thrift-server-0.3.7.jar:na]
> Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
> Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
>
>
>
> On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:
>
>> Did you check if packets are NOT being dropped for network interfaces
>> Cassandra instances are consuming (ifconfig –a) internode compression is
>> set for all endpoint – may be network is playing any role here?
>>
>> is this corruption limited so certain keyspace/table | DCs or is that
>> wide spread – the log snippet you shared it looked like only specific
>> keyspace/table is affected – is that correct?
>>
>> When you remove corrupted sstable of a certain table, I guess you
>> verifies all nodes for corrupted sstables for same table (may be with with
>> nodetool scrub tool) so to limit spread of corruptions – right?
>>
>> Just curious to know – you’re not using lz4/default compressor for all
>> tables there must be some reason for it.
>>
>>
>>
>>
>>
>>
>>
>> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
>> *Sent:* Thursday, August 08, 2019 6:20 AM
>> *To:* user@cassandra.apache.org
>> *Subject:* Re: Datafile Corruption
>>
>>
>>
>> Hi All,
>>
>> Thank you so much for the replies.
>>
>> Currently, I have the following list that can potentially cause some sort
>> of corruption in a Cassandra cluster.
>>
>>    - Sudden Power cut  -  *We have had no power cuts in the datacenters*
>>    - Network Issues - *no network issues from what I can tell*
>>    - Disk full - *I don't think this is an issue for us, see disks
>>    below.*
>>    - An issue in Casandra version like Cassandra-13752 -* couldn't find
>>    any Jira issues similar to ours.*
>>    - Bit Flips -* we have compression enabled so I don't think this
>>    should be an issue.*
>>    - Repair during upgrade has caused corruption too -* we have not
>>    upgraded*
>>    - Dropping and adding columns with the same name but a different type
>>    - *I will need to ask the apps team how they are using the database.*
>>
>>
>>
>> Ok, let me try and explain the issue we are having, I am under a lot of
>> pressure from above to get this fixed and I can't figure it out.
>>
>> This is a PRE-PROD environment.
>>
>>    - 2 datacenters.
>>    - 9 physical servers in each datacenter
>>    - 4 Cassandra instances on each server
>>    - 72 Cassandra instances across the 2 data centres, 36 in site A, 36
>>    in site B.
>>
>>
>> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
>> datacenter each running with its own Cassandra back end in a cluster
>> together.
>>
>> OS Details [Red Hat Linux]
>> cass_a@x 0 10:53:01 ~ $ uname -a
>> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
>> x86_64 x86_64 x86_64 GNU/Linux
>>
>> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
>> NAME="Red Hat Enterprise Linux Server"
>> VERSION="7.6 (Maipo)"
>> ID="rhel"
>>
>> Storage Layout
>> cass_a@xx 0 10:46:28 ~ $ df -h
>> Filesystem                         Size  Used Avail Use% Mounted on
>> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
>> devtmpfs                            63G     0   63G   0% /dev
>> tmpfs                               63G     0   63G   0% /dev/shm
>> tmpfs                               63G  4.1G   59G   7% /run
>> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>> >> 4 cassandra instances
>> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
>> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
>> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
>> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>>
>> Cassandra load is about 200GB and the rest of the space is snapshots
>>
>> CPU
>> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
>> CPU(s):                64
>> Thread(s) per core:    2
>> Core(s) per socket:    16
>> Socket(s):             2
>>
>> *Description of problem:*
>> During repair of the cluster, we are seeing multiple corruptions in the
>> log files on a lot of instances.  There seems to be no pattern to the
>> corruption.  It seems that the repair job is finding all the corrupted
>> files for us.  The repair will hang on the node where the corrupted file is
>> found.  To fix this we remove/rename the datafile and bounce the Cassandra
>> instance.  Our hardware/OS team have stated there is no problem on their
>> side.  I do not believe it the repair causing the corruption.
>>
>> We have maintenance scripts that run every night running compactions and
>> creating snapshots, I decided to turn these off, fix any corruptions we
>> currently had and ran major compaction on all nodes, once this was done we
>> had a "clean" cluster and we left the cluster for a few days.  After the
>> process we noticed one corruption in the cluster, this datafile was created
>> after I turned off the maintenance scripts so my theory of the scripts
>> causing the issue was wrong.  We then kicked off another repair and started
>> to find more corrupt files created after the maintenance script was turned
>> off.
>>
>>
>> So let me give you an example of a corrupted file and maybe someone might
>> be able to work through it with me?
>>
>> When this corrupted file was reported in the log it looks like it was the
>> repair that found it.
>>
>> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
>> --until "2019-08-07 22:45:00"
>>
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
>> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle
>> tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
>> KeyspaceMetadata/x, (-1476350953672479093,-1474461
>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
>> Thread[ValidationExecutor:825,1,main]
>> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>> ~[guava-16.0.jar:na]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.repair.Validator.add(Validator.java:150)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>> ~[na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>> [na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: at
>> java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
>> /data/ssd2/data/KeyspaceMetadata/x-x/l
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
>> Aug 07 22:30:33 cassandra[34611]: Caused by:
>> org.apache.cassandra.io.compress.CorruptBlockException:
>> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
>> Aug 07 22:30:33 cassandra[34611]: at
>> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
>> ~[apache-cassandra-2.2.13.jar:2.2.13]
>> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair,
>> will not do anticompaction
>> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
>> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
>> operator request
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
>> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
>> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
>> state jump to shutdown
>>
>>
>> So I went to the file system to see when this corrupt file was created
>> and it was created on July 30th at 15.55
>>
>> root@x 0 01:14:03 ~ # ls -l
>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55
>> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>>
>>
>>
>> So I checked /var/log/messages for errors around that time
>> The only thing that stands out to me is the message "Cannot perform a
>> full major compaction as repaired and unrepaired sstables cannot be
>> compacted together", I'm not sure if this would be an issue though and
>> cause corruption.
>>
>> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
>> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
>> Jul 30 15:55:06 x audispd: node=x. type=USER_START
>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
>> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
>> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
>> ses=4294967295 msg='cwd="/"
>> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
>> terminal=? res=success'
>> Jul 30 15:55:14 x tag_audit_log: type=USER_START
>> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
>> compaction as repaired and unrepaired sstables cannot be compacted
>> together. These two set of sstables will be compacted separately.
>> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
>> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes,
>> 57 ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
>> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>> terminal=/dev/pts/0 res=success'
>> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
>> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
>> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
>> terminal=/dev/pts/1 res=success'
>>
>> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
>> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
>> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
>> Jul 30 15:55:57 x audispd: node=x. type=USER_START
>> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
>> ses=4294967295 msg='op=PAM:session_open
>> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
>> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
>> res=success'
>> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes, 1
>> ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
>> compaction as repaired and unrepaired sstables cannot be compacted
>> together. These two set of sstables will be compacted separately.
>> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
>> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes,
>> 16 ops, 0%/0% of on/off-heap limit)
>> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
>> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
>> ses=4294967295 msg='cwd="/"
>> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
>> terminal=? res=success'
>>
>>
>>
>> We have checked a number of other things like NTP setting etc but nothing
>> is telling us what could cause so many corruptions across the entire
>> cluster.
>> Things were healthy with this cluster for months, the only thing I can
>> think is that we started loading data from a load of 20GB per instance up
>> to 200GB where it sits now, maybe this just highlighted the issue.
>>
>>
>>
>> Compaction and Compression on Keyspace CL's [mixture]
>> All CF's are using compression.
>>
>> AND compaction = {'min_threshold': '4', 'class':
>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>> 'max_threshold': '32'}
>> AND compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>
>> AND compaction = {'min_threshold': '4', 'class':
>> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
>> 'max_threshold': '32'}
>> AND compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>>
>> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
>> *LeveledCompactionStrategy*'}
>> AND compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>>
>> --We are also using internode network compression:
>> internode_compression: all
>>
>>
>>
>> Does anyone have any idea what I should check next?
>> Our next theory is that there may be an issue with Checksum, but I'm not
>> sure where to go with this.
>>
>>
>>
>> Any help would be very much appreciated before I lose the last bit of
>> hair I have on my head.
>>
>>
>>
>> Kind Regards,
>>
>> Phil
>>
>>
>>
>> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>>
>> Repair during upgrade have caused corruption too.
>>
>>
>>
>> Also, dropping and adding columns with same name but different type
>>
>>
>>
>> Regards,
>>
>> Nitan
>>
>> Cell: 510 449 9629
>>
>>
>> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>>
>> Is compression enabled?
>>
>>
>>
>> If not, bit flips on disk can corrupt data files and reads + repair may
>> send that corruption to other hosts in the cluster
>>
>>
>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
>> wrote:
>>
>> Hi All,
>>
>>
>>
>> I am currently experiencing multiple datafile corruptions across most
>> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
>> starting to think it might be a bug, we're using Cassandra 2.2.13.
>>
>>
>>
>> Without going into detail about the issue I just want to confirm
>> something.
>>
>>
>>
>> Can someone share with me a list of scenarios that would cause corruption?
>>
>>
>>
>> 1. OS failure
>>
>> 2. Cassandra disturbed during the writing
>>
>>
>>
>> etc etc.
>>
>>
>>
>> I need to investigate each scenario and don't want to leave any out.
>>
>>
>>
>> --
>>
>> Regards,
>>
>> Phil
>>
>>
>>
>>
>> --
>>
>> Regards,
>>
>> Phil
>>
>
>
> --
> Regards,
> Phil
>

Re: Datafile Corruption

Posted by Philip Ó Condúin <ph...@gmail.com>.
*@Jeff *- If it was hardware that would explain it all, but do you think
it's possible to have every server in the cluster with a hardware issue?
The data is sensitive and the customer would lose their mind if I sent it
off-site which is a pity cause I could really do with the help.
The corruption is occurring irregularly on every server and instance and
column family in the cluster.  Out of 72 instances, we are getting maybe 10
corrupt files per day.
We are using vnodes (256) and it is happening in both DC's

*@Asad *- internode compression is set to ALL on every server.  I have
checked the packets for the private interconnect and I can't see any
dropped packets, there are dropped packets for other interfaces, but not
for the private ones, I will get the network team to double-check this.
The corruption is only on the application schema, we are not getting
corruption on any system or cass keyspaces.  Corruption is happening in
both DC's.  We are getting corruption for the 1 application schema we have
across all tables in the keyspace, it's not limited to one table.
Im not sure why the app team decided to not use default compression, I must
ask them.



I have been checking the /var/log/messages today going back a few weeks and
can see a serious amount of broken pipe errors across all servers and
instances.
Here is a snippet from one server but most pipe errors are similar:

Jul  9 03:00:08  cassandra: INFO  02:00:08 Writing
Memtable-sstable_activity@1126262628(43.631KiB serialized bytes, 18072 ops,
0%/0% of on/off-heap limit)
Jul  9 03:00:13  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:19  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:22  cassandra: ERROR 02:00:22 Got an IOException during write!
Jul  9 03:00:22  cassandra: java.io.IOException: Broken pipe
Jul  9 03:00:22  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native
Method) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at
sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at
sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at
sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
~[na:1.8.0_172]
Jul  9 03:00:22  cassandra: at
org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
~[libthrift-0.9.2.jar:0.9.2]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.Message.write(Message.java:222)
~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:22  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:25  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:30  cassandra: ERROR 02:00:30 Got an IOException during write!
Jul  9 03:00:30  cassandra: java.io.IOException: Broken pipe
Jul  9 03:00:30  cassandra: at sun.nio.ch.FileDispatcherImpl.write0(Native
Method) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at
sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at
sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) ~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at sun.nio.ch.IOUtil.write(IOUtil.java:65)
~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at
sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471)
~[na:1.8.0_172]
Jul  9 03:00:30  cassandra: at
org.apache.thrift.transport.TNonblockingSocket.write(TNonblockingSocket.java:165)
~[libthrift-0.9.2.jar:0.9.2]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.util.mem.Buffer.writeTo(Buffer.java:104)
~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.util.mem.FastMemoryOutputTransport.streamTo(FastMemoryOutputTransport.java:112)
~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.Message.write(Message.java:222)
~[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.handleWrite(TDisruptorServer.java:598)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.processKey(TDisruptorServer.java:569)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.select(TDisruptorServer.java:423)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:30  cassandra: at
com.thinkaurelius.thrift.TDisruptorServer$AbstractSelectorThread.run(TDisruptorServer.java:383)
[thrift-server-0.3.7.jar:na]
Jul  9 03:00:31  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:37  kernel: fnic_handle_fip_timer: 8 callbacks suppressed
Jul  9 03:00:43  kernel: fnic_handle_fip_timer: 8 callbacks suppressed



On Thu, 8 Aug 2019 at 15:42, ZAIDI, ASAD A <az...@att.com> wrote:

> Did you check if packets are NOT being dropped for network interfaces
> Cassandra instances are consuming (ifconfig –a) internode compression is
> set for all endpoint – may be network is playing any role here?
>
> is this corruption limited so certain keyspace/table | DCs or is that wide
> spread – the log snippet you shared it looked like only specific
> keyspace/table is affected – is that correct?
>
> When you remove corrupted sstable of a certain table, I guess you verifies
> all nodes for corrupted sstables for same table (may be with with nodetool
> scrub tool) so to limit spread of corruptions – right?
>
> Just curious to know – you’re not using lz4/default compressor for all
> tables there must be some reason for it.
>
>
>
>
>
>
>
> *From:* Philip Ó Condúin [mailto:philipoconduin@gmail.com]
> *Sent:* Thursday, August 08, 2019 6:20 AM
> *To:* user@cassandra.apache.org
> *Subject:* Re: Datafile Corruption
>
>
>
> Hi All,
>
> Thank you so much for the replies.
>
> Currently, I have the following list that can potentially cause some sort
> of corruption in a Cassandra cluster.
>
>    - Sudden Power cut  -  *We have had no power cuts in the datacenters*
>    - Network Issues - *no network issues from what I can tell*
>    - Disk full - *I don't think this is an issue for us, see disks below.*
>    - An issue in Casandra version like Cassandra-13752 -* couldn't find
>    any Jira issues similar to ours.*
>    - Bit Flips -* we have compression enabled so I don't think this
>    should be an issue.*
>    - Repair during upgrade has caused corruption too -* we have not
>    upgraded*
>    - Dropping and adding columns with the same name but a different type
>    - *I will need to ask the apps team how they are using the database.*
>
>
>
> Ok, let me try and explain the issue we are having, I am under a lot of
> pressure from above to get this fixed and I can't figure it out.
>
> This is a PRE-PROD environment.
>
>    - 2 datacenters.
>    - 9 physical servers in each datacenter
>    - 4 Cassandra instances on each server
>    - 72 Cassandra instances across the 2 data centres, 36 in site A, 36
>    in site B.
>
>
> We also have 2 Reaper Nodes we use for repair.  One reaper node in each
> datacenter each running with its own Cassandra back end in a cluster
> together.
>
> OS Details [Red Hat Linux]
> cass_a@x 0 10:53:01 ~ $ uname -a
> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
> x86_64 x86_64 x86_64 GNU/Linux
>
> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
> NAME="Red Hat Enterprise Linux Server"
> VERSION="7.6 (Maipo)"
> ID="rhel"
>
> Storage Layout
> cass_a@xx 0 10:46:28 ~ $ df -h
> Filesystem                         Size  Used Avail Use% Mounted on
> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
> devtmpfs                            63G     0   63G   0% /dev
> tmpfs                               63G     0   63G   0% /dev/shm
> tmpfs                               63G  4.1G   59G   7% /run
> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
> >> 4 cassandra instances
> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
>
> Cassandra load is about 200GB and the rest of the space is snapshots
>
> CPU
> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
> CPU(s):                64
> Thread(s) per core:    2
> Core(s) per socket:    16
> Socket(s):             2
>
> *Description of problem:*
> During repair of the cluster, we are seeing multiple corruptions in the
> log files on a lot of instances.  There seems to be no pattern to the
> corruption.  It seems that the repair job is finding all the corrupted
> files for us.  The repair will hang on the node where the corrupted file is
> found.  To fix this we remove/rename the datafile and bounce the Cassandra
> instance.  Our hardware/OS team have stated there is no problem on their
> side.  I do not believe it the repair causing the corruption.
>
> We have maintenance scripts that run every night running compactions and
> creating snapshots, I decided to turn these off, fix any corruptions we
> currently had and ran major compaction on all nodes, once this was done we
> had a "clean" cluster and we left the cluster for a few days.  After the
> process we noticed one corruption in the cluster, this datafile was created
> after I turned off the maintenance scripts so my theory of the scripts
> causing the issue was wrong.  We then kicked off another repair and started
> to find more corrupt files created after the maintenance script was turned
> off.
>
>
> So let me give you an example of a corrupted file and maybe someone might
> be able to work through it with me?
>
> When this corrupted file was reported in the log it looks like it was the
> repair that found it.
>
> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
> --until "2019-08-07 22:45:00"
>
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
> Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle
> tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
> KeyspaceMetadata/x, (-1476350953672479093,-1474461
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
> Thread[ValidationExecutor:825,1,main]
> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
> /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.repair.Validator.add(Validator.java:150)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> ~[na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> [na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at java.lang.Thread.run(Thread.java:748)
> [na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: Caused by:
> org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
> /data/ssd2/data/KeyspaceMetadata/x-x/l
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
> Aug 07 22:30:33 cassandra[34611]: Caused by:
> org.apache.cassandra.io.compress.CorruptBlockException:
> (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
> Aug 07 22:30:33 cassandra[34611]: at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
> ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair, will
> not do anticompaction
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
> operator request
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=>
> state jump to shutdown
>
>
> So I went to the file system to see when this corrupt file was created and
> it was created on July 30th at 15.55
>
> root@x 0 01:14:03 ~ # ls -l
> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55
> /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
>
>
>
> So I checked /var/log/messages for errors around that time
> The only thing that stands out to me is the message "Cannot perform a full
> major compaction as repaired and unrepaired sstables cannot be compacted
> together", I'm not sure if this would be an issue though and cause
> corruption.
>
> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
> Jul 30 15:55:06 x audispd: node=x. type=USER_START
> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
> msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
> ses=4294967295 msg='cwd="/"
> cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
> terminal=? res=success'
> Jul 30 15:55:14 x tag_audit_log: type=USER_START
> msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
> Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
> compaction as repaired and unrepaired sstables cannot be compacted
> together. These two set of sstables will be compacted separately.
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
> Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes, 57
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
> Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
> msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
> terminal=/dev/pts/0 res=success'
> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
> msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
> msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
> terminal=/dev/pts/1 res=success'
>
> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
> Jul 30 15:55:57 x audispd: node=x. type=USER_START
> msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
> acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:57 x audispd: node=x. type=USER_START
> msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
> ses=4294967295 msg='op=PAM:session_open
> grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
> acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
> res=success'
> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
> Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes, 1
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
> compaction as repaired and unrepaired sstables cannot be compacted
> together. These two set of sstables will be compacted separately.
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
> Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes, 16
> ops, 0%/0% of on/off-heap limit)
> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
> msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
> ses=4294967295 msg='cwd="/"
> cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
> terminal=? res=success'
>
>
>
> We have checked a number of other things like NTP setting etc but nothing
> is telling us what could cause so many corruptions across the entire
> cluster.
> Things were healthy with this cluster for months, the only thing I can
> think is that we started loading data from a load of 20GB per instance up
> to 200GB where it sits now, maybe this just highlighted the issue.
>
>
>
> Compaction and Compression on Keyspace CL's [mixture]
> All CF's are using compression.
>
> AND compaction = {'min_threshold': '4', 'class':
> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
> 'max_threshold': '32'}
> AND compression = {'sstable_compression':
> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>
> AND compaction = {'min_threshold': '4', 'class':
> 'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
> 'max_threshold': '32'}
> AND compression = {'sstable_compression':
> 'org.apache.cassandra.io.compress.*LZ4Compressor*'}
>
> AND compaction = {'class': 'org.apache.cassandra.db.compaction.
> *LeveledCompactionStrategy*'}
> AND compression = {'sstable_compression':
> 'org.apache.cassandra.io.compress.*SnappyCompressor*'}
>
> --We are also using internode network compression:
> internode_compression: all
>
>
>
> Does anyone have any idea what I should check next?
> Our next theory is that there may be an issue with Checksum, but I'm not
> sure where to go with this.
>
>
>
> Any help would be very much appreciated before I lose the last bit of hair
> I have on my head.
>
>
>
> Kind Regards,
>
> Phil
>
>
>
> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>
> Repair during upgrade have caused corruption too.
>
>
>
> Also, dropping and adding columns with same name but different type
>
>
>
> Regards,
>
> Nitan
>
> Cell: 510 449 9629
>
>
> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>
> Is compression enabled?
>
>
>
> If not, bit flips on disk can corrupt data files and reads + repair may
> send that corruption to other hosts in the cluster
>
>
> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
> wrote:
>
> Hi All,
>
>
>
> I am currently experiencing multiple datafile corruptions across most
> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
> starting to think it might be a bug, we're using Cassandra 2.2.13.
>
>
>
> Without going into detail about the issue I just want to confirm something.
>
>
>
> Can someone share with me a list of scenarios that would cause corruption?
>
>
>
> 1. OS failure
>
> 2. Cassandra disturbed during the writing
>
>
>
> etc etc.
>
>
>
> I need to investigate each scenario and don't want to leave any out.
>
>
>
> --
>
> Regards,
>
> Phil
>
>
>
>
> --
>
> Regards,
>
> Phil
>


-- 
Regards,
Phil

RE: Datafile Corruption

Posted by "ZAIDI, ASAD A" <az...@att.com>.
Did you check if packets are NOT being dropped for network interfaces Cassandra instances are consuming (ifconfig –a) internode compression is set for all endpoint – may be network is playing any role here?
is this corruption limited so certain keyspace/table | DCs or is that wide spread – the log snippet you shared it looked like only specific keyspace/table is affected – is that correct?
When you remove corrupted sstable of a certain table, I guess you verifies all nodes for corrupted sstables for same table (may be with with nodetool scrub tool) so to limit spread of corruptions – right?
Just curious to know – you’re not using lz4/default compressor for all tables there must be some reason for it.



From: Philip Ó Condúin [mailto:philipoconduin@gmail.com]
Sent: Thursday, August 08, 2019 6:20 AM
To: user@cassandra.apache.org
Subject: Re: Datafile Corruption

Hi All,

Thank you so much for the replies.

Currently, I have the following list that can potentially cause some sort of corruption in a Cassandra cluster.

  *   Sudden Power cut  -  We have had no power cuts in the datacenters
  *   Network Issues - no network issues from what I can tell
  *   Disk full - I don't think this is an issue for us, see disks below.
  *   An issue in Casandra version like Cassandra-13752 - couldn't find any Jira issues similar to ours.
  *   Bit Flips - we have compression enabled so I don't think this should be an issue.
  *   Repair during upgrade has caused corruption too - we have not upgraded
  *   Dropping and adding columns with the same name but a different type - I will need to ask the apps team how they are using the database.


Ok, let me try and explain the issue we are having, I am under a lot of pressure from above to get this fixed and I can't figure it out.

This is a PRE-PROD environment.

  *   2 datacenters.
  *   9 physical servers in each datacenter
  *   4 Cassandra instances on each server
  *   72 Cassandra instances across the 2 data centres, 36 in site A, 36 in site B.

We also have 2 Reaper Nodes we use for repair.  One reaper node in each datacenter each running with its own Cassandra back end in a cluster together.

OS Details [Red Hat Linux]
cass_a@x 0 10:53:01 ~ $ uname -a
Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018 x86_64 x86_64 x86_64 GNU/Linux

cass_a@x 0 10:57:31 ~ $ cat /etc/*release
NAME="Red Hat Enterprise Linux Server"
VERSION="7.6 (Maipo)"
ID="rhel"

Storage Layout
cass_a@xx 0 10:46:28 ~ $ df -h
Filesystem                         Size  Used Avail Use% Mounted on
/dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
devtmpfs                            63G     0   63G   0% /dev
tmpfs                               63G     0   63G   0% /dev/shm
tmpfs                               63G  4.1G   59G   7% /run
tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>> 4 cassandra instances
/dev/sdd                           1.5T  802G  688G  54% /data/ssd4
/dev/sda                           1.5T  798G  692G  54% /data/ssd1
/dev/sdb                           1.5T  681G  810G  46% /data/ssd2
/dev/sdc                           1.5T  558G  932G  38% /data/ssd3

Cassandra load is about 200GB and the rest of the space is snapshots

CPU
cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
CPU(s):                64
Thread(s) per core:    2
Core(s) per socket:    16
Socket(s):             2

Description of problem:
During repair of the cluster, we are seeing multiple corruptions in the log files on a lot of instances.  There seems to be no pattern to the corruption.  It seems that the repair job is finding all the corrupted files for us.  The repair will hang on the node where the corrupted file is found.  To fix this we remove/rename the datafile and bounce the Cassandra instance.  Our hardware/OS team have stated there is no problem on their side.  I do not believe it the repair causing the corruption.

We have maintenance scripts that run every night running compactions and creating snapshots, I decided to turn these off, fix any corruptions we currently had and ran major compaction on all nodes, once this was done we had a "clean" cluster and we left the cluster for a few days.  After the process we noticed one corruption in the cluster, this datafile was created after I turned off the maintenance scripts so my theory of the scripts causing the issue was wrong.  We then kicked off another repair and started to find more corrupt files created after the maintenance script was turned off.


So let me give you an example of a corrupted file and maybe someone might be able to work through it with me?

When this corrupted file was reported in the log it looks like it was the repair that found it.

$ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00" --until "2019-08-07 22:45:00"

Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing Memtable-compactions_in_progress@830377457(0.008KiB<mailto:Memtable-compactions_in_progress@830377457(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on KeyspaceMetadata/x, (-1476350953672479093,-1474461
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread Thread[ValidationExecutor:825,1,main]
Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.Iterators$7.computeNext(Iterators.java:645) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.repair.Validator.rowHash(Validator.java:201) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.repair.Validator.add(Validator.java:150) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /data/ssd2/data/KeyspaceMetadata/x-x/l
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
Aug 07 22:30:33 cassandra[34611]: Caused by: org.apache.cassandra.io.compress.CorruptBlockException: (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185) ~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair, will not do anticompaction
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by operator request
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37<https://urldefense.proofpoint.com/v2/url?u=http-3A__10.2.57.37&d=DwMFaQ&c=LFYZ-o9_HUMeMTSQicvjIg&r=FsmDztdsVuIKml8IDhdHdg&m=4S7F10IxFntsiwIo-XT-YrkZE8312_yG8jMeOqOBjpE&s=20PLv0KNcUpBbyY1byoboimsLRjbPCLV76xL37jpttQ&e=> state jump to shutdown


So I went to the file system to see when this corrupt file was created and it was created on July 30th at 15.55

root@x 0 01:14:03 ~ # ls -l /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
-rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db



So I checked /var/log/messages for errors around that time
The only thing that stands out to me is the message "Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together", I'm not sure if this would be an issue though and cause corruption.

Jul 30 15:55:06 x systemd: Created slice User Slice of root.
Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
Jul 30 15:55:06 x audispd: node=x. type=USER_START msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
Jul 30 15:55:14 x tag_audit_log: type=USER_CMD msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295 ses=4294967295 msg='cwd="/" cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473 terminal=? res=success'
Jul 30 15:55:14 x tag_audit_log: type=USER_START msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing Memtable-compactions_in_progress@1462227999(0.008KiB<mailto:Memtable-compactions_in_progress@1462227999(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing Memtable-compactions_in_progress@1198535528(1.002KiB<mailto:Memtable-compactions_in_progress@1198535528(1.002KiB> serialized bytes, 57 ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing Memtable-compactions_in_progress@2039409834(0.008KiB<mailto:Memtable-compactions_in_progress@2039409834(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747 msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=? terminal=/dev/pts/0 res=success'
Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749 msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=? terminal=/dev/pts/1 res=success'

Jul 30 15:55:57 x systemd: Created slice User Slice of root.
Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
Jul 30 15:55:57 x audispd: node=x. type=USER_START msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:57 x audispd: node=x. type=USER_START msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing Memtable-compactions_in_progress@1532791194(0.008KiB<mailto:Memtable-compactions_in_progress@1532791194(0.008KiB> serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing Memtable-compactions_in_progress@1455399453(0.281KiB<mailto:Memtable-compactions_in_progress@1455399453(0.281KiB> serialized bytes, 16 ops, 0%/0% of on/off-heap limit)
Jul 30 15:56:04 x tag_audit_log: type=USER_CMD msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295 ses=4294967295 msg='cwd="/" cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473 terminal=? res=success'



We have checked a number of other things like NTP setting etc but nothing is telling us what could cause so many corruptions across the entire cluster.
Things were healthy with this cluster for months, the only thing I can think is that we started loading data from a load of 20GB per instance up to 200GB where it sits now, maybe this just highlighted the issue.



Compaction and Compression on Keyspace CL's [mixture]
All CF's are using compression.

AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}

AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.LZ4Compressor'}

AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}

--We are also using internode network compression:
internode_compression: all



Does anyone have any idea what I should check next?
Our next theory is that there may be an issue with Checksum, but I'm not sure where to go with this.

Any help would be very much appreciated before I lose the last bit of hair I have on my head.

Kind Regards,
Phil

On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com>> wrote:
Repair during upgrade have caused corruption too.

Also, dropping and adding columns with same name but different type

Regards,
Nitan
Cell: 510 449 9629<tel:510%20449%209629>

On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com>> wrote:
Is compression enabled?

If not, bit flips on disk can corrupt data files and reads + repair may send that corruption to other hosts in the cluster

On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>> wrote:
Hi All,

I am currently experiencing multiple datafile corruptions across most nodes in my cluster, there seems to be no pattern to the corruption.  I'm starting to think it might be a bug, we're using Cassandra 2.2.13.

Without going into detail about the issue I just want to confirm something.

Can someone share with me a list of scenarios that would cause corruption?

1. OS failure
2. Cassandra disturbed during the writing

etc etc.

I need to investigate each scenario and don't want to leave any out.

--
Regards,
Phil


--
Regards,
Phil

Re: Datafile Corruption

Posted by Jeff Jirsa <jj...@gmail.com>.
The corrupt block exception from the compressor in 2.1/2.2 is something I don’t recall ever being attributed to anything other than bad hardware, so that seems by far the most likely option. 

The corruption that the compressor is catching says the checksum written immediately after the compressed block doesn’t match when read back. 

How sensitive is this data? Would you be able to send one of the corrupt data files to a developer to check? Or is it something like PII you can’t share? 

Have you found this corruption on every single instance? Are you using single tokens or vnodes? Is it happening in both dcs? 




> On Aug 8, 2019, at 4:20 AM, Philip Ó Condúin <ph...@gmail.com> wrote:
> 
> Hi All,
> 
> Thank you so much for the replies.  
> 
> Currently, I have the following list that can potentially cause some sort of corruption in a Cassandra cluster. 
> 
> Sudden Power cut  -  We have had no power cuts in the datacenters
> Network Issues - no network issues from what I can tell
> Disk full - I don't think this is an issue for us, see disks below.
> An issue in Casandra version like Cassandra-13752 - couldn't find any Jira issues similar to ours.
> Bit Flips - we have compression enabled so I don't think this should be an issue.
> Repair during upgrade has caused corruption too - we have not upgraded
> Dropping and adding columns with the same name but a different type - I will need to ask the apps team how they are using the database.
> 
> 
> Ok, let me try and explain the issue we are having, I am under a lot of pressure from above to get this fixed and I can't figure it out.
> 
> This is a PRE-PROD environment.
> 2 datacenters.
> 9 physical servers in each datacenter
> 4 Cassandra instances on each server
> 72 Cassandra instances across the 2 data centres, 36 in site A, 36 in site B.
> 
> We also have 2 Reaper Nodes we use for repair.  One reaper node in each datacenter each running with its own Cassandra back end in a cluster together.
> 
> OS Details [Red Hat Linux]
> cass_a@x 0 10:53:01 ~ $ uname -a
> Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018 x86_64 x86_64 x86_64 GNU/Linux
> 
> cass_a@x 0 10:57:31 ~ $ cat /etc/*release
> NAME="Red Hat Enterprise Linux Server"
> VERSION="7.6 (Maipo)"
> ID="rhel"
> 
> Storage Layout 
> cass_a@xx 0 10:46:28 ~ $ df -h
> Filesystem                         Size  Used Avail Use% Mounted on
> /dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
> devtmpfs                            63G     0   63G   0% /dev
> tmpfs                               63G     0   63G   0% /dev/shm
> tmpfs                               63G  4.1G   59G   7% /run
> tmpfs                               63G     0   63G   0% /sys/fs/cgroup
> >> 4 cassandra instances
> /dev/sdd                           1.5T  802G  688G  54% /data/ssd4
> /dev/sda                           1.5T  798G  692G  54% /data/ssd1
> /dev/sdb                           1.5T  681G  810G  46% /data/ssd2
> /dev/sdc                           1.5T  558G  932G  38% /data/ssd3
> 
> Cassandra load is about 200GB and the rest of the space is snapshots
> 
> CPU
> cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
> CPU(s):                64
> Thread(s) per core:    2
> Core(s) per socket:    16
> Socket(s):             2
> 
> Description of problem:
> During repair of the cluster, we are seeing multiple corruptions in the log files on a lot of instances.  There seems to be no pattern to the corruption.  It seems that the repair job is finding all the corrupted files for us.  The repair will hang on the node where the corrupted file is found.  To fix this we remove/rename the datafile and bounce the Cassandra instance.  Our hardware/OS team have stated there is no problem on their side.  I do not believe it the repair causing the corruption. 
> 
> We have maintenance scripts that run every night running compactions and creating snapshots, I decided to turn these off, fix any corruptions we currently had and ran major compaction on all nodes, once this was done we had a "clean" cluster and we left the cluster for a few days.  After the process we noticed one corruption in the cluster, this datafile was created after I turned off the maintenance scripts so my theory of the scripts causing the issue was wrong.  We then kicked off another repair and started to find more corrupt files created after the maintenance script was turned off.
> 
> 
> So let me give you an example of a corrupted file and maybe someone might be able to work through it with me?
> 
> When this corrupted file was reported in the log it looks like it was the repair that found it.
> 
> $ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00" --until "2019-08-07 22:45:00"
> 
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on KeyspaceMetadata/x, (-1476350953672479093,-1474461
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread Thread[ValidationExecutor:825,1,main]
> Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /x/ssd2/data/KeyspaceMetadata/x-1e453cb0
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.Iterators$7.computeNext(Iterators.java:645) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.repair.Validator.rowHash(Validator.java:201) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.repair.Validator.add(Validator.java:150) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: at java.lang.Thread.run(Thread.java:748) [na:1.8.0_172]
> Aug 07 22:30:33 cassandra[34611]: Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /data/ssd2/data/KeyspaceMetadata/x-x/l
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
> Aug 07 22:30:33 cassandra[34611]: Caused by: org.apache.cassandra.io.compress.CorruptBlockException: (/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
> Aug 07 22:30:33 cassandra[34611]: at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185) ~[apache-cassandra-2.2.13.jar:2.2.13]
> Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair, will not do anticompaction
> Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
> Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by operator request
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
> Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37 state jump to shutdown
> 
> 
> So I went to the file system to see when this corrupt file was created and it was created on July 30th at 15.55
> 
> root@x 0 01:14:03 ~ # ls -l /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
> -rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55 /data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
> 
> 
> 
> So I checked /var/log/messages for errors around that time
> The only thing that stands out to me is the message "Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together", I'm not sure if this would be an issue though and cause corruption.
> 
> Jul 30 15:55:06 x systemd: Created slice User Slice of root.
> Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
> Jul 30 15:55:06 x audispd: node=x. type=USER_START msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
> Jul 30 15:55:14 x tag_audit_log: type=USER_CMD msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295 ses=4294967295 msg='cwd="/" cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473 terminal=? res=success'
> Jul 30 15:55:14 x tag_audit_log: type=USER_START msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.
> Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes, 57 ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747 msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=? terminal=/dev/pts/0 res=success'
> Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749 msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=? terminal=/dev/pts/1 res=success'
> 
> Jul 30 15:55:57 x systemd: Created slice User Slice of root.
> Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
> Jul 30 15:55:57 x audispd: node=x. type=USER_START msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:57 x audispd: node=x. type=USER_START msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295 ses=4294967295 msg='op=PAM:session_open grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=? res=success'
> Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes, 1 ops, 0%/0% of on/off-heap limit)
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.
> Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes, 16 ops, 0%/0% of on/off-heap limit)
> Jul 30 15:56:04 x tag_audit_log: type=USER_CMD msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295 ses=4294967295 msg='cwd="/" cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473 terminal=? res=success'
> 
> 
> 
> We have checked a number of other things like NTP setting etc but nothing is telling us what could cause so many corruptions across the entire cluster.
> Things were healthy with this cluster for months, the only thing I can think is that we started loading data from a load of 20GB per instance up to 200GB where it sits now, maybe this just highlighted the issue.
> 
> 
> 
> Compaction and Compression on Keyspace CL's [mixture]
> All CF's are using compression.
> 
> AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32'}
> AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}
> 
> AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32'}
> AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.LZ4Compressor'}
> 
> AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'}
> AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}
> 
> --We are also using internode network compression:
> internode_compression: all
> 
> 
> 
> Does anyone have any idea what I should check next?
> Our next theory is that there may be an issue with Checksum, but I'm not sure where to go with this.
> 
> Any help would be very much appreciated before I lose the last bit of hair I have on my head. 
> 
> Kind Regards,
> Phil
> 
>> On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:
>> Repair during upgrade have caused corruption too.
>> 
>> Also, dropping and adding columns with same name but different type
>> 
>> 
>> Regards,
>> Nitan
>> Cell: 510 449 9629
>> 
>>> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>>> 
>>> Is compression enabled?
>>> 
>>> If not, bit flips on disk can corrupt data files and reads + repair may send that corruption to other hosts in the cluster 
>>> 
>>> 
>>>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com> wrote:
>>>> 
>>>> Hi All,
>>>> 
>>>> I am currently experiencing multiple datafile corruptions across most nodes in my cluster, there seems to be no pattern to the corruption.  I'm starting to think it might be a bug, we're using Cassandra 2.2.13.
>>>> 
>>>> Without going into detail about the issue I just want to confirm something.
>>>> 
>>>> Can someone share with me a list of scenarios that would cause corruption?
>>>> 
>>>> 1. OS failure
>>>> 2. Cassandra disturbed during the writing 
>>>> 
>>>> etc etc.
>>>> 
>>>> I need to investigate each scenario and don't want to leave any out.
>>>> 
>>>> -- 
>>>> Regards,
>>>> Phil
> 
> 
> -- 
> Regards,
> Phil

Re: Datafile Corruption

Posted by Philip Ó Condúin <ph...@gmail.com>.
Hi All,

Thank you so much for the replies.

Currently, I have the following list that can potentially cause some sort
of corruption in a Cassandra cluster.


   - Sudden Power cut  -  *We have had no power cuts in the datacenters*
   - Network Issues - *no network issues from what I can tell*
   - Disk full - *I don't think this is an issue for us, see disks below.*
   - An issue in Casandra version like Cassandra-13752 -* couldn't find any
   Jira issues similar to ours.*
   - Bit Flips -* we have compression enabled so I don't think this should
   be an issue.*
   - Repair during upgrade has caused corruption too -* we have not
   upgraded*
   - Dropping and adding columns with the same name but a different type - *I
   will need to ask the apps team how they are using the database.*



Ok, let me try and explain the issue we are having, I am under a lot of
pressure from above to get this fixed and I can't figure it out.

This is a PRE-PROD environment.

   - 2 datacenters.
   - 9 physical servers in each datacenter
   - 4 Cassandra instances on each server
   - 72 Cassandra instances across the 2 data centres, 36 in site A, 36 in
   site B.


We also have 2 Reaper Nodes we use for repair.  One reaper node in each
datacenter each running with its own Cassandra back end in a cluster
together.

OS Details [Red Hat Linux]
cass_a@x 0 10:53:01 ~ $ uname -a
Linux x 3.10.0-957.5.1.el7.x86_64 #1 SMP Wed Dec 19 10:46:58 EST 2018
x86_64 x86_64 x86_64 GNU/Linux

cass_a@x 0 10:57:31 ~ $ cat /etc/*release
NAME="Red Hat Enterprise Linux Server"
VERSION="7.6 (Maipo)"
ID="rhel"

Storage Layout
cass_a@xx 0 10:46:28 ~ $ df -h
Filesystem                         Size  Used Avail Use% Mounted on
/dev/mapper/vg01-lv_root            20G  2.2G   18G  11% /
devtmpfs                            63G     0   63G   0% /dev
tmpfs                               63G     0   63G   0% /dev/shm
tmpfs                               63G  4.1G   59G   7% /run
tmpfs                               63G     0   63G   0% /sys/fs/cgroup
>> 4 cassandra instances
/dev/sdd                           1.5T  802G  688G  54% /data/ssd4
/dev/sda                           1.5T  798G  692G  54% /data/ssd1
/dev/sdb                           1.5T  681G  810G  46% /data/ssd2
/dev/sdc                           1.5T  558G  932G  38% /data/ssd3

Cassandra load is about 200GB and the rest of the space is snapshots

CPU
cass_a@x 127 10:58:47 ~ $ lscpu | grep -E '^Thread|^Core|^Socket|^CPU\('
CPU(s):                64
Thread(s) per core:    2
Core(s) per socket:    16
Socket(s):             2

*Description of problem:*
During repair of the cluster, we are seeing multiple corruptions in the log
files on a lot of instances.  There seems to be no pattern to the
corruption.  It seems that the repair job is finding all the corrupted
files for us.  The repair will hang on the node where the corrupted file is
found.  To fix this we remove/rename the datafile and bounce the Cassandra
instance.  Our hardware/OS team have stated there is no problem on their
side.  I do not believe it the repair causing the corruption.

We have maintenance scripts that run every night running compactions and
creating snapshots, I decided to turn these off, fix any corruptions we
currently had and ran major compaction on all nodes, once this was done we
had a "clean" cluster and we left the cluster for a few days.  After the
process we noticed one corruption in the cluster, this datafile was created
after I turned off the maintenance scripts so my theory of the scripts
causing the issue was wrong.  We then kicked off another repair and started
to find more corrupt files created after the maintenance script was turned
off.


So let me give you an example of a corrupted file and maybe someone might
be able to work through it with me?

When this corrupted file was reported in the log it looks like it was the
repair that found it.

$ journalctl -u cassmeta-cass_b.service --since "2019-08-07 22:25:00"
--until "2019-08-07 22:45:00"

Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Writing
Memtable-compactions_in_progress@830377457(0.008KiB serialized bytes, 1
ops, 0%/0% of on/off-heap limit)
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Failed creating a merkle
tree for [repair #9587a200-b95a-11e9-8920-9f72868b8375 on
KeyspaceMetadata/x, (-1476350953672479093,-1474461
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Exception in thread
Thread[ValidationExecutor:825,1,main]
Aug 07 22:30:33 cassandra[34611]: org.apache.cassandra.io.FSReadError:
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
/x/ssd2/data/KeyspaceMetadata/x-1e453cb0
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:365)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:361)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:340)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:382)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.composites.AbstractCType$Serializer.deserialize(AbstractCType.java:366)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:81)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:52)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.AbstractCell$1.computeNext(AbstractCell.java:46)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:169)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext(MergeIterator.java:202)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.Iterators$7.computeNext(Iterators.java:645)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
~[guava-16.0.jar:na]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(ColumnIndex.java:174)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.compaction.LazilyCompactedRow.update(LazilyCompactedRow.java:187)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.repair.Validator.rowHash(Validator.java:201)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.repair.Validator.add(Validator.java:150)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1166)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:76)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.db.compaction.CompactionManager$10.call(CompactionManager.java:736)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
~[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: at java.lang.Thread.run(Thread.java:748)
[na:1.8.0_172]
Aug 07 22:30:33 cassandra[34611]: Caused by:
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
/data/ssd2/data/KeyspaceMetadata/x-x/l
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:216)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:226)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer(CompressedThrottledReader.java:42)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:352)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: ... 27 common frames omitted
Aug 07 22:30:33 cassandra[34611]: Caused by:
org.apache.cassandra.io.compress.CorruptBlockException:
(/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big
Aug 07 22:30:33 cassandra[34611]: at
org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBufferMmap(CompressedRandomAccessReader.java:185)
~[apache-cassandra-2.2.13.jar:2.2.13]
Aug 07 22:30:33 cassandra[34611]: ... 30 common frames omitted
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Not a global repair, will
not do anticompaction
Aug 07 22:30:33 cassandra[34611]: ERROR 21:30:33 Stopping gossiper
Aug 07 22:30:33 cassandra[34611]: WARN  21:30:33 Stopping gossip by
operator request
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Announcing shutdown
Aug 07 22:30:33 cassandra[34611]: INFO  21:30:33 Node /10.2.57.37 state
jump to shutdown


So I went to the file system to see when this corrupt file was created and
it was created on July 30th at 15.55

root@x 0 01:14:03 ~ # ls -l
/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db
-rw-r--r-- 1 cass_b cass_b 3182243670 Jul 30 15:55
/data/ssd2/data/KeyspaceMetadata/x-x/lb-26203-big-Data.db



So I checked /var/log/messages for errors around that time
The only thing that stands out to me is the message "Cannot perform a full
major compaction as repaired and unrepaired sstables cannot be compacted
together", I'm not sure if this would be an issue though and cause
corruption.

Jul 30 15:55:06 x systemd: Created slice User Slice of root.
Jul 30 15:55:06 x systemd: Started Session c165280 of user root.
Jul 30 15:55:06 x audispd: node=x. type=USER_START
msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
ses=4294967295 msg='op=PAM:session_open
grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:06 x systemd: Removed slice User Slice of root.
Jul 30 15:55:14 x tag_audit_log: type=USER_CMD
msg=audit(1564498506.013:457932): pid=17533 uid=509 auid=4294967295
ses=4294967295 msg='cwd="/"
cmd=2F7573722F7362696E2F69706D692D73656E736F7273202D2D71756965742D6361636865202D2D7364722D63616368652D7265637265617465202D2D696E746572707265742D6F656D2D64617461202D2D6F75747075742D73656E736F722D7374617465202D2D69676E6F72652D6E6F742D617661696C61626C652D73656E736F7273202D2D6F75747075742D73656E736F722D7468726573686F6C6473
terminal=? res=success'
Jul 30 15:55:14 x tag_audit_log: type=USER_START
msg=audit(1564498506.021:457933): pid=17533 uid=0 auid=4294967295
ses=4294967295 msg='op=PAM:session_open
grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
Memtable-compactions_in_progress@1462227999(0.008KiB serialized bytes, 1
ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Cannot perform a full major
compaction as repaired and unrepaired sstables cannot be compacted
together. These two set of sstables will be compacted separately.
Jul 30 15:55:19 x cassandra: INFO  14:55:19 Writing
Memtable-compactions_in_progress@1198535528(1.002KiB serialized bytes, 57
ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:20 x cassandra: INFO  14:55:20 Writing
Memtable-compactions_in_progress@2039409834(0.008KiB serialized bytes, 1
ops, 0%/0% of on/off-heap limit)
Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
msg=audit(1564498524.409:457934): pid=46620 uid=0 auid=464400029 ses=2747
msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
terminal=/dev/pts/0 res=success'
Jul 30 15:55:24 x audispd: node=x. type=USER_LOGOUT
msg=audit(1564498524.409:457935): pid=4878 uid=0 auid=464400029 ses=2749
msg='op=login id=464400029 exe="/usr/sbin/sshd" hostname=? addr=?
terminal=/dev/pts/1 res=success'

Jul 30 15:55:57 x systemd: Created slice User Slice of root.
Jul 30 15:55:57 x systemd: Started Session c165288 of user root.
Jul 30 15:55:57 x audispd: node=x. type=USER_START
msg=audit(1564498557.294:457958): pid=19687 uid=0 auid=4294967295
ses=4294967295 msg='op=PAM:session_open
grantors=pam_keyinit,pam_limits,pam_keyinit,pam_limits,pam_tty_audit,pam_systemd,pam_unix
acct="root" exe="/usr/bin/sudo" hostname=? addr=? terminal=? res=success'
Jul 30 15:55:57 x audispd: node=x. type=USER_START
msg=audit(1564498557.298:457959): pid=19690 uid=0 auid=4294967295
ses=4294967295 msg='op=PAM:session_open
grantors=pam_keyinit,pam_systemd,pam_keyinit,pam_limits,pam_unix
acct="cass_b" exe="/usr/sbin/runuser" hostname=? addr=? terminal=?
res=success'
Jul 30 15:55:58 x systemd: Removed slice User Slice of root.
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
Memtable-compactions_in_progress@1532791194(0.008KiB serialized bytes, 1
ops, 0%/0% of on/off-heap limit)
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Cannot perform a full major
compaction as repaired and unrepaired sstables cannot be compacted
together. These two set of sstables will be compacted separately.
Jul 30 15:56:02 x cassandra: INFO  14:56:02 Writing
Memtable-compactions_in_progress@1455399453(0.281KiB serialized bytes, 16
ops, 0%/0% of on/off-heap limit)
Jul 30 15:56:04 x tag_audit_log: type=USER_CMD
msg=audit(1564498555.190:457951): pid=19294 uid=509 auid=4294967295
ses=4294967295 msg='cwd="/"
cmd=72756E75736572202D73202F62696E2F62617368202D6C20636173735F62202D632063617373616E6472612D6D6574612F63617373616E6472612F62696E2F6E6F6465746F6F6C2074707374617473
terminal=? res=success'



We have checked a number of other things like NTP setting etc but nothing
is telling us what could cause so many corruptions across the entire
cluster.
Things were healthy with this cluster for months, the only thing I can
think is that we started loading data from a load of 20GB per instance up
to 200GB where it sits now, maybe this just highlighted the issue.



Compaction and Compression on Keyspace CL's [mixture]
All CF's are using compression.

AND compaction = {'min_threshold': '4', 'class':
'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
'max_threshold': '32'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.
*SnappyCompressor*'}

AND compaction = {'min_threshold': '4', 'class':
'org.apache.cassandra.db.compaction.*SizeTieredCompactionStrategy*',
'max_threshold': '32'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.
*LZ4Compressor*'}

AND compaction = {'class': 'org.apache.cassandra.db.compaction.
*LeveledCompactionStrategy*'}
AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.
*SnappyCompressor*'}

--We are also using internode network compression:
internode_compression: all



Does anyone have any idea what I should check next?
Our next theory is that there may be an issue with Checksum, but I'm not
sure where to go with this.

Any help would be very much appreciated before I lose the last bit of hair
I have on my head.

Kind Regards,
Phil

On Wed, 7 Aug 2019 at 20:51, Nitan Kainth <ni...@gmail.com> wrote:

> Repair during upgrade have caused corruption too.
>
> Also, dropping and adding columns with same name but different type
>
>
> Regards,
>
> Nitan
>
> Cell: 510 449 9629
>
> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
>
> Is compression enabled?
>
> If not, bit flips on disk can corrupt data files and reads + repair may
> send that corruption to other hosts in the cluster
>
>
> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com>
> wrote:
>
> Hi All,
>
> I am currently experiencing multiple datafile corruptions across most
> nodes in my cluster, there seems to be no pattern to the corruption.  I'm
> starting to think it might be a bug, we're using Cassandra 2.2.13.
>
> Without going into detail about the issue I just want to confirm something.
>
> Can someone share with me a list of scenarios that would cause corruption?
>
> 1. OS failure
> 2. Cassandra disturbed during the writing
>
> etc etc.
>
> I need to investigate each scenario and don't want to leave any out.
>
> --
> Regards,
> Phil
>
>

-- 
Regards,
Phil

Re: Datafile Corruption

Posted by Nitan Kainth <ni...@gmail.com>.
Repair during upgrade have caused corruption too.

Also, dropping and adding columns with same name but different type


Regards,
Nitan
Cell: 510 449 9629

> On Aug 7, 2019, at 2:42 PM, Jeff Jirsa <jj...@gmail.com> wrote:
> 
> Is compression enabled?
> 
> If not, bit flips on disk can corrupt data files and reads + repair may send that corruption to other hosts in the cluster 
> 
> 
>> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com> wrote:
>> 
>> Hi All,
>> 
>> I am currently experiencing multiple datafile corruptions across most nodes in my cluster, there seems to be no pattern to the corruption.  I'm starting to think it might be a bug, we're using Cassandra 2.2.13.
>> 
>> Without going into detail about the issue I just want to confirm something.
>> 
>> Can someone share with me a list of scenarios that would cause corruption?
>> 
>> 1. OS failure
>> 2. Cassandra disturbed during the writing 
>> 
>> etc etc.
>> 
>> I need to investigate each scenario and don't want to leave any out.
>> 
>> -- 
>> Regards,
>> Phil

Re: Datafile Corruption

Posted by Jeff Jirsa <jj...@gmail.com>.
Is compression enabled?

If not, bit flips on disk can corrupt data files and reads + repair may send that corruption to other hosts in the cluster 


> On Aug 7, 2019, at 3:46 AM, Philip Ó Condúin <ph...@gmail.com> wrote:
> 
> Hi All,
> 
> I am currently experiencing multiple datafile corruptions across most nodes in my cluster, there seems to be no pattern to the corruption.  I'm starting to think it might be a bug, we're using Cassandra 2.2.13.
> 
> Without going into detail about the issue I just want to confirm something.
> 
> Can someone share with me a list of scenarios that would cause corruption?
> 
> 1. OS failure
> 2. Cassandra disturbed during the writing 
> 
> etc etc.
> 
> I need to investigate each scenario and don't want to leave any out.
> 
> -- 
> Regards,
> Phil