Hi all,

we experience the following issue when executing full sequential repairs in 
Cassandra 4.0.10.

ERROR [RepairSnapshotExecutor:1] 2023-11-07 13:22:50,267 
CassandraDaemon.java:581 - Exception in thread 
Thread[RepairSnapshotExecutor:1,5,main]
java.lang.RuntimeException: Tried to create duplicate hard link to 
/opt/ddb/data/pool/data1/test_keyspace/test1-c4b33340f0a211edb0cb2fb04a4be304/snapshots/bec3dba0-7d70-11ee-99d3-7bda513c2b90/nb-1-big-Filter.db
        at 
org.apache.cassandra.io.util.FileUtils.createHardLink(FileUtils.java:185)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.createLinks(SSTableReader.java:1624)
        at 
org.apache.cassandra.io.sstable.format.SSTableReader.createLinks(SSTableReader.java:1606)
        at 
org.apache.cassandra.db.ColumnFamilyStore.snapshotWithoutFlush(ColumnFamilyStore.java:1852)
        at 
org.apache.cassandra.db.ColumnFamilyStore.snapshot(ColumnFamilyStore.java:2031)
        at 
org.apache.cassandra.db.ColumnFamilyStore.snapshot(ColumnFamilyStore.java:2017)
        at 
org.apache.cassandra.db.repair.CassandraTableRepairManager.lambda$snapshot$0(CassandraTableRepairManager.java:74)
        at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
        at java.util.concurrent.FutureTask.run(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
        at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(Unknown Source)
ERROR [AntiEntropyStage:1] 2023-11-07 13:22:50,267 CassandraDaemon.java:581 - 
Exception in thread Thread[AntiEntropyStage:1,5,main]
java.lang.RuntimeException: java.lang.RuntimeException: Unable to take a 
snapshot bec3dba0-7d70-11ee-99d3-7bda513c2b90 on test_keyspace/test1
This behavior is reproduced consistently, when the following are true:
*       It is a normal sequential repair (--full and --sequential),
*       It is not a global repair, meaning at least one datacenter is defined 
(--in-dc or --in-local-dc),
*       The repair affects more than two Cassandra nodes.

For more than two Cassandra nodes the parent repair session consists of 
multiple separate repair sessions towards different target endpoints. Full 
sequential repairs require that all participants flush and snapshot the data 
before starting the repair. Unfortunately, there is a collision between the 
separate repair sessions. The first one creates the ephemeral snapshot 
successfully and the second one that tries to create a snapshot (create hard 
link) in the same node fails with the above error.
This issue is not seen in global repairs, where datacenters and hosts are not 
defined, because in that case there is an explicit check if a snapshot already 
exists before proceeding.

I found a few issues in Jira about duplicate hard links, but all of them are 
from older versions and seem irrelevant to this one. Could you please help with 
this issue?

Thank you,
Panagiotis


Reply via email to