[
https://issues.apache.org/jira/browse/CASSANDRA-20490?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17952086#comment-17952086
]
Stefan Miklosovic commented on CASSANDRA-20490:
-----------------------------------------------
this timeouts even before CASSANDRA-18119
org.apache.cassandra.io.sstable.SSTableReaderTest testSpannedIndexPositions
{code}
INFO [main] 2025-05-16T11:50:51,576 ColumnFamilyStore.java:3025 - Truncation
is dropping 1 sstables and keeping 0 due to sstable.maxDataAge > truncatedAt
INFO [NonPeriodicTasks:1] 2025-05-16T11:50:51,579 BigFormat.java:324 -
Deleting sstable:
/home/fermat/dev/cassandra/cassandra-instaclustr/cassandra/build/test/cassandra/data/SSTableReaderTest/Standard1-02ffd7b0324c11f0b6ac1dd07b28a950/oa-5-big
INFO [NonPeriodicTasks:1] 2025-05-16T11:50:51,582 LogTransaction.java:254 -
Unfinished transaction log, deleting
/home/fermat/dev/cassandra/cassandra-instaclustr/cassandra/build/test/cassandra/data/SSTableReaderTest/Standard1-02ffd7b0324c11f0b6ac1dd07b28a950/oa_txn_unknowncompactiontype_04134380-324c-11f0-b6ac-1dd07b28a950.log
INFO [main] 2025-05-16T11:50:51,641 ColumnFamilyStore.java:1052 - Enqueuing
flush of SSTableReaderTest.Standard1, Reason: UNIT_TESTS, Usage: 3.018MiB (0%)
on-heap, 0B (0%) off-heap
INFO [PerDiskMemtableFlushWriter_0:2] 2025-05-16T11:50:51,647
Flushing.java:153 - Writing Memtable-Standard1@1339346383(244.141KiB serialized
bytes, 5000 ops, 3.018MiB (0%) on-heap, 0B (0%) off-heap), flushed range =
[null, null)
INFO [PerDiskMemtableFlushWriter_0:2] 2025-05-16T11:50:51,681
Flushing.java:179 - Completed flushing
/home/fermat/dev/cassandra/cassandra-instaclustr/cassandra/build/test/cassandra/data/SSTableReaderTest/Standard1-02ffd7b0324c11f0b6ac1dd07b28a950/oa-6-big-Data.db
(82.403KiB) for commitlog position CommitLogPosition(segmentId=1747396247710,
position=29104)
ERROR [Reference-Reaper] 2025-05-16T11:50:51,723 Ref.java:250 - LEAK DETECTED:
a reference (class
org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@269773909:/home/fermat/dev/cassandra/cassandra-instaclustr/cassandra/build/test/cassandra/data/SSTableReaderTest/StandardSmallBloomFilter-030073fa324c11f0b6ac1dd07b28a950/oa-2-big)
to class
org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@269773909:/home/fermat/dev/cassandra/cassandra-instaclustr/cassandra/build/test/cassandra/data/SSTableReaderTest/StandardSmallBloomFilter-030073fa324c11f0b6ac1dd07b28a950/oa-2-big
was not released before the reference was garbage collected
{code}
this started to fail upon CASSANDRA-18119
{code}
org.apache.cassandra.db.lifecycle.LogTransactionTest
testStatsTSMismatchDuringStart
org.apache.cassandra.db.lifecycle.LogTransactionTest testWrongTimestampInTxnFile
org.apache.cassandra.db.lifecycle.LogTransactionTest
testStatsTSMismatchDuringList
{code}
this is https://issues.apache.org/jira/browse/CASSANDRA-20307
org.apache.cassandra.fuzz.sai.MultiNodeSAITest indexOnlySaiTest
This is already reported in
https://issues.apache.org/jira/browse/CASSANDRA-20417 but I do not not see
related tickets.
{code}
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.MigrationFromAccordReadRaceTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.MigrationFromAccordReadRaceTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.MigrationFromAccordWriteRaceTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.MigrationFromAccordWriteRaceTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.MigrationToAccordWriteRaceTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.MigrationToAccordWriteRaceTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.AccordMigrationTest
junit.framework.TestSuite
org.apache.cassandra.distributed.test.accord.AccordMigrationTest
{code}
refresh_test.TestRefresh test_refresh_deadlock_startup is
https://issues.apache.org/jira/browse/CASSANDRA-20306
> Encountred "duplicate hardlink error" when repairing
> ----------------------------------------------------
>
> Key: CASSANDRA-20490
> URL: https://issues.apache.org/jira/browse/CASSANDRA-20490
> Project: Apache Cassandra
> Issue Type: Bug
> Components: Consistency/Repair
> Reporter: Stefan Miklosovic
> Assignee: Stefan Miklosovic
> Priority: Normal
> Fix For: 5.1
>
> Time Spent: 2h 10m
> Remaining Estimate: 0h
>
> A user reported:
> 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
> https://lists.apache.org/thread/kwz89po5gkx68bhof7l7o0yykz48bnbw
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]