IBM Support

system.log:org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:

Troubleshooting


Problem

 

*Article should supplement https://support.datastax.com/s/article/Validation-failed-when-running-a-nodetool-repair

https://blog.pythian.com/so-you-have-a-broken-cassandra-sstable-file/

Overview

This technical note addresses failing repairs and corrupt SSTable exception found in system.log


Symptom

When a background repair occurs or a repair command, nodetool repair is run, the following error is encountered suggesting that the repair has failed:

ERROR: Error running task 0x450ea7: Some repair failed (MainThread)

 

Another error suggesting the same failed repair can be:

error: Repair job has failed with the error message: [2020-08-11 14:44:52,173] Some repair failed
-- StackTrace --
java.lang.RuntimeException: Repair job has failed with the error message: [2020-08-11 14:44:52,173] Some repair failed


Analysis

There is a visible error with the repair, but it is only a symptom of another problem. It is recommended to look at system.log to give more visibility on the actual issue:

system.log:org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: some_example.db

 

There are cases where CorruptSSTableException is not explicitly mentioned in system.log. One example can be,

ERROR [RepairJobTask:8] 2018-08-08 15:15:57,726 RepairRunnable.java:277 - Repair session 2c5f89e0-9b39-11e8-b5ee-bb8feee1767a for range [(-1377105920845202291,-1371711029446682941], (-8865445607623519086,-885162575564883.... 425683885]]] Sync failed between /192.168.1.90 and /192.168.1.92
/var/log/cassandra/debug.log:ERROR [RepairJobTask:4] 2018-08-09 16:16:50,722 RepairSession.java:281 - [repair #25682740-9c11-11e8-8e8f-fbc0ff4d2cb8] Session completed with the following error
/var/log/cassandra/debug.log:ERROR [RepairJobTask:4] 2018-08-09 16:16:50,726 RepairRunnable.java:277 - Repair session 25682740-9c11-11e8-8e8f-fbc0ff4d2cb8...... 7115161941975432804,7115472305105341673], (5979423340500726528,5980417142425683885]]] Validation failed in /192.168.1.88
/var/log/cassandra/system.log:ERROR [ValidationExecutor:2] 2018-08-09 16:16:50,707 Validator.java:268 - Failed creating a merkle tree for [repair #25682740-9c11-11e8-8e8f-fbc0ff4d2cb8 on keyspace1/standard1, ...

 

The initial error message that a Sync failure had occurred can be misleading, but looking further at the next few error lines might give more clues. From the last snippet, Failed creating a merkle tree for [repair #25682740-9c11-11e8-8e8f-fbc0ff4d2cb8 on keyspace1/standard1, ... suggests that ‘keyspace1’ is broken which is likely to be the main cause. 


What is a corrupt SSTable?

SSTables are usually corrupt because they fail an internal consistency check such as a column length is too long and/or checksum validation. A corrupted SSTable file does not mean data is lost or the cluster is unusable so long as the Replication Factor (RF) is set to the recommended three or more.


How does Corrupt SSTable affect performances?

Corrupt SSTables have relatively little effect on normal reads against the table except for the request where the failure took place. However, It has a serious effect on compactions and repair, and may prevent these processes from completing.

Repair failure can result in long-term consistency issues between nodes and eventually the application returning incorrect results. Compaction failure may cause the number of SSTables to grow uncontrollably. In the short term, read performance will be adversely affected and in the long term, storage space problems will surface.


Solution

Once the problem is identified as a corrupt SSTable, there are four different solutions to fix the problem with different results and risks:

  1. nodetool scrub - online scrub. This means that this option can be done while the node is UP.  Relatively lower chance of success. Potentially repeating the repair issue.
  2. rm -f   -  offline operation. Remove the SSTable while the node is offline. nodetool repair immediately after node is brought up. Fastest and easiest with some consistency risks when bringing up.
  3. Bootstrap the node - Similar to #3 with less theoretical impact on consistency.
  4. nodetool sstablescrub - offline scrub operation. This means the node has to be brought down before running sstablescrub. Much higher chance of success than online scrub. Requires a long time ( >48 hours) to complete for a decently sized SSTable (20MB and above). Offline scrub should be the last resort.


Option 1 - nodetool scrub

https://docs.datastax.com/en/dse/5.1/dse-admin/datastax_enterprise/tools/nodetool/toolsScrub.html

Perform the following steps to complete an online scrub. Easy to perform, but with a relatively low chance of success:

  1. Find out which SSTable is broken.
  2. Run nodetool scrub keyspace tablename.
  3. Run nodetool repair -full.
  4. Run nodetool listsnapshots.
  5. Run nodetool clearsnapshot keyspacename -t snapshot name.

Corrupt SSTable is usually stated in the error message found in system.log. Follow the steps in Analysis to identify which SSTable is broken.

$ nodetool scrub keyspace1 standard1

 

Nodetool scrub will snapshot and rebuild the files. Some and possibly all rows from corrupt SSTable will disappear which will be repaired by $ nodetool repair. There will be fewer SSTables and file names might have changed. 


Perform repair to recover files

$ nodetool repair -full

 

The output should look similar to the following:

[2018-08-09 17:00:51,663] Starting repair command #2 (4c820390-9c17-11e8-8e8f-fbc0ff4d2cb8), repairing keyspace keyspace1 with repair options (parallelism: parallel, primary range: false, incremental: false, job threads: 1, ColumnFamilies: [], dataCenters: [], hosts: [], # of ranges: 768, pull repair: false)
[2018-08-09 18:14:09,799] Repair session 4cadf590-9c17-11e8-8e8f-fbc0ff4d2cb8 for range [(-1377105920845202291,...
[2018-08-09 18:14:10,130] Repair completed successfully [2018-08-09 18:14:10,131] Repair command #2 finished in 1 hour 13 minutes 18 seconds

 

After a successful repair, a new snapshot called pre-scrub-<timestamp> is created. To reduce disk space usage, this snapshot can be removed. Start by listing snapshots:

$ nodetool listsnapshots

 

One of the snapshots will be:

pre-scrub-1533897462847 keyspace1 standard1 35.93 GiB 35.93 GiB

 

Clear the snapshot:

$ nodetool clearsnapshot -t pre-scrub-1533897462847

 

If repair still fails to complete and the validation error persists, one of the other three methods should be attempted.


Option 2 - Delete the files and run nodetool repair

If the node is down which renders online scrub impossible to do, it is recommended to just remove the corrupted file from the directory. This method should work 100% of the time and much faster than offline scrub. The time estimate for the entire process should be minutes. This solution is highly recommended for a node that is down due to corruption and is required to be up immediately.


Removing just one corrupted SSTable might not allow the down node to fully restart. If there are multiple corrupted SSTables, the node will fail to boot up on the first identifiable corrupt SSTable. Hence, the log will only show one corrupt SSTable exception for that one table but not for other corrupt SSTables.  All corrupt SSTables must be removed before a node can be fully started. 


Users should consider the time tradeoff between manually deleting the corrupted files one by one as it appears versus bootstrapping the entire node. When there are too many corrupted sstables, it is highly recommended to perform Option 3 - Bootstrapping instead. 


Warning: When the Consistency Level (CL) is set to ONE, there is an increased risk of read consistency. However, it is uncommon and not recommended for CL to be set to ONE.


Steps:

  1. Bring the node down with nodetool drain.
  2. Navigate to the corrupted keyspace and sstable directory. This is usually at /var/lib/cassandra/data/
  3. Delete the specific corrupt SSTable. If unable to identify the specific ones, delete all files in the directory with sudo rm -f *
  4. Restart the node.
  5. Run nodetool repair.


Bring down the node following the steps in Option 2:

$ nodetool drain
$ pkill cassandra


Navigate to the corrupted keyspace and SSTable directory, for example:

$ cd /var/lib/cassandra/data/keyspace1/standard1-afd416808c7311e8a0c96796602809/


Delete the specific sstable if the specific table can be identified. In this example, the entire files are deleted: 

$ sudo rm -f *


The output will be:

rm: cannot remove 'backups': Is a directory
rm: cannot remove 'snapshots': Is a directory


When checking the remaining files you will see backups and snapshots:

$ ls
backups snapshots 

 

Restart the node:

$ systemctl start cassandra

 

After a node is fully started, run nodetool repair on the directory immediately:

$ nodetool repair keyspace1 standard1 -full
[2018-08-10 11:23:22,454] Starting repair command #1 (51713c00-9cb1-11e8-ba61-01c8f56621df), repairing keyspace keyspace1 with repair options (parallelism: parallel, primary range: false, incremental: false, job threads: 1, ColumnFamilies: [standard1], dataCenters: [], hosts: [], # of ranges: 768, pull repair: false)
[2018-08-10 13:02:36,097] Repair completed successfully
[2018-08-10 13:02:36,098] Repair command #1 finished in 1 hour 39 minutes 13 seconds

 

Option 3 - Bootstrap the node

Bootstrap should be performed when too many corrupted sstables are present in the node and file deletion process proves to be too cumbersome. Bootstrapping will mean that nodes containing missing data will not be read until all data is restored. 


Warning: Bootstrap can operate in parallel, but depending on the amount of data that has to be recovered, this method can take longer than other options.


Steps:

  1. Bring down the node with $ nodetool drain.
  2. Remove all files under $CASSANDRA_HOME. This is usually var/lib/Cassandra.
  3. Modify Apache Cassandra environment in /etc/cassandra/conf/cassandra-env.sh.
  4. Restart Apache Cassandra. Server starting with no files will stream data from all nodes to one of its seeds to replace the lost data.
  5. Modify Apache Cassandra environment in /etc/cassandra/conf/cassandra-env.sh file to undo the change introduced in Step 3.

Bring down the node following the steps in Option 2:

$ nodetool drain
$ sudo pkill java

 

Modify Apache Cassandra environment:

$ vi /etc/cassandra/conf/cassandra-env.sh

 

Add this line at the end of the file:

JVM_OPTS="$JVM_OPTS -Dcassandra.replace_address=192.168.1.88″

 

192.168.1.88 is the address the Apache Cassandra service is on. Upon restarting, the server will connect to one of the seeds. The server will try to recreate the schema by requesting all nodes to stream data to that seed and replace the lost data. 


New token ranges will not be selected unless the service is restarted with a different IP than before. Hence, the edit in the environment file specifies the old address where the repair failure happened at 192.168.1.88.


Restart the cluster:

$ systemctl start cassandra


Wait for the node to join the cluster. Bootstrap occurs when the message appears:

INFO [main] 2018-08-10 13:39:06,780 StreamResultFuture.java:90 - [Stream #47b382f0-9cc4-11e8-a010-51948a7598a1] Executing streaming plan for Bootstrap
INFO [StreamConnectionEstablisher:1] 2018-08-10 13:39:06,784 StreamSession.java:266 - [Stream #47b382f0-9cc4-11e8-a010-51948a7598a1] Starting streaming to /192.168.1.90 >/code>


The output message will eventually be:

INFO [main] 2018-08-10 14:18:16,133 StorageService.java:1449 - JOINING: Finish joining ring
INFO [main] 2018-08-10 14:18:16,482 SecondaryIndexManager.java:509 - Executing pre-join post-bootstrap tasks for: CFS(Keyspace='keyspace1′, ColumnFamily='standard1′)
INFO [main] 2018-08-10 14:18:16,484 SecondaryIndexManager.java:509 - Executing pre-join post-bootstrap tasks for: CFS(Keyspace='keyspace1′, ColumnFamily='counter1′)
INFO [main] 2018-08-10 14:18:16,897 StorageService.java:2292 - Node /192.168.1.88 state jump to NORMAL
WARN [main] 2018-08-10 14:18:16,899 StorageService.java:2324 - Not updating token metadata for /192.168.1.88 because I am replacing it



Option 4 - sstablescrub

https://docs.datastax.com/en/dse/6.8/dse-admin/datastax_enterprise/tools/toolsSStables/toolsSSTableScrub.html


Offline sstablescrub has better success rate than its online version, nodetool scrub. Only attempt when other methods fail and are impossible to do. This is especially true for when your RF is one. SSTable scrub will require many hours and most of the time, days to complete for even a medium sized SSTable (20 MB and above). 


Warning: Never use the offline scrub method that is sstablescrub for the entire node. Offline scrubbing an entire node will take days because this method has to scrub and rebuild ALL tables in the node. SSTablescrub should always be targeted specific to a keyspace and table.


Offline sstablescrub is a last resort solution.


Steps:

  1. Bring the node down with $ nodetool drain.
  2. Run $ sstablescrub
  3. Restart the node
  4. Run $ nodetool repair on the table
  5. Run $ nodetool clearsnapshot to remove pre-scrub snapshot


Run the following commands to bring the node down:

$ nodetool drain
$ pkill cassandra


sstablescrub even with -n option can take days to complete a scrub for 1GiB sstable. Sstable scrub is not realistic for most situations unless SSTable size is very small. 


sstablescrub can be run by:

$ sstablescrub -n keyspace1 standard1


Once sstablescrub completes, restart the node

$ systemctl start cassandra 


Run the repair command:

$ nodetool repair keyspace1 standard1 -full


Delete the pre-scrub snapshot

$ nodetool clearsnapshot -t pre-scrub-<timestamp>


Flowchart

Presented with a corrupt sstable, the online nodetool scrub should always be attempted first because it is the easiest and safest solution and offline sstablescrub should be attempted last due to the process being extremely slow. Depending on the use cases, we can follow the flowchart below when trying to fix SSTable corruption:

flowchart

 


False Corrupt SSTable Exception

Sometimes Corrupt SSTable Exception can occur due to AIO memory issues preventing the SSTable from being deserialized and read properly. In this case the SSTable is not corrupted, it was just not read successfully. You will see Caused by: messages in the stacktrace similar to the following:

...
Caused by: java.io.IOException: Error building row with data deserialized from RandomAccessReader: {rebufferer=Prefetching rebufferer: (8/4) buffers read-ahead, 4096 buffer size buffer=java.nio.DirectByteBuffer[pos=4096 lim=4096 cap=4096] bufferHolder=org.apache.cassandra.io.util.WrappingRebufferer$WrappingBufferHolder@2a6640b9}
    at org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:641)
    at org.apache.cassandra.db.UnfilteredDeserializer.readNext(UnfilteredDeserializer.java:168)
    at org.apache.cassandra.io.sstable.format.AbstractReader.readUnfiltered(AbstractReader.java:257)
    at org.apache.cassandra.io.sstable.format.trieindex.ForwardReader.nextInSlice(ForwardReader.java:55)
    at org.apache.cassandra.io.sstable.format.AbstractReader.next(AbstractReader.java:138)
    at org.apache.cassandra.io.sstable.format.AsyncPartitionReader$PartitionSubscription.performRead(AsyncPartitionReader.java:537)
    at org.apache.cassandra.io.sstable.format.AsyncPartitionReader.readWithRetry(AsyncPartitionReader.java:251)
    ... 31 common frames omitted
Caused by: org.apache.cassandra.io.sstable.BufferPoolException: Failed to allocate address nr. 0 of size 4096: buffer pool is probably exhausted, consider setting file_cache_size_in_mb and inflight_data_overhead_in_mb in the yaml
    at org.apache.cassandra.utils.memory.buffers.PermanentBufferPool.allocate(PermanentBufferPool.java:144)
    at org.apache.cassandra.cache.ChunkCacheImpl.newChunk(ChunkCacheImpl.java:607)
    at org.apache.cassandra.cache.ChunkCacheImpl.asyncLoad(ChunkCacheImpl.java:652)
    at org.apache.cassandra.cache.ChunkCacheImpl.asyncLoad(ChunkCacheImpl.java:67)
    at com.github.benmanes.caffeine.cache.LocalAsyncLoadingCache.lambda$get$2(LocalAsyncLoadingCache.java:129)
    at com.github.benmanes.caffeine.cache.LocalCache.lambda$statsAware

In prior DSE 6.x (such as 6.7) releases, DataStax recommended disabling AIO and setting file_cache_size_in_mb to 512 for search workloads, to improve indexing and query performance.


See the following link for details:
https://docs.datastax.com/en/dse/6.8/docs/search/tune-index.html#EnablingAsynchronousI/O(AIO)

Document Location

Worldwide

[{"Type":"MASTER","Line of Business":{"code":"LOB76","label":"Data Platform"},"Business Unit":{"code":"BU048","label":"IBM Software"},"Product":{"code":"SSQWIX","label":"DataStax Luna"},"ARM Category":[{"code":"","label":""}],"ARM Case Number":"","Platform":[{"code":"PF025","label":"Platform Independent"}],"Version":"All Version(s)"}]

Historical Number

ka06R000000Hc3SQAS

Document Information

Modified date:
30 January 2026

UID

ibm17258927