I am wondering if the cause is tarring when creating hardlinks, i.e. creating a new snapshot.
A quick experiment on my Mac indicates the file status (ctime) is updated when creating hardlink. *➜ *stat -f "Access (atime): %Sa%nModify (mtime): %Sm%nChange (ctime): %Sc" a Access (atime): Mar 22 10:03:43 2022 Modify (mtime): Mar 22 10:03:43 2022 Change (ctime): Mar 22 10:05:43 2022 On Tue, Mar 22, 2022 at 10:01 AM Jeff Jirsa <jji...@gmail.com> wrote: > The most useful thing that folks can provide is an indication of what was > writing to those data files when you were doing backups. > > It's almost certainly one of: > - Memtable flush > - Compaction > - Streaming from repair/move/bootstrap > > If you have logs that indicate compaction starting/finishing with those > sstables, or memtable flushing those sstables, or if the .log file is > included in your backup, pasting the contents of that .log file into a > ticket will make this much easier to debug. > > > > On Tue, Mar 22, 2022 at 9:49 AM Yifan Cai <yc25c...@gmail.com> wrote: > >> I do not think there is a ticket already. Feel free to create one. >> https://issues.apache.org/jira/projects/CASSANDRA/issues/ >> >> It would be helpful to provide >> 1. The version of the cassandra >> 2. The options used for snapshotting >> >> - Yifan >> >> On Tue, Mar 22, 2022 at 9:41 AM Paul Chandler <p...@redshots.com> wrote: >> >>> Hi all, >>> >>> Was there any further progress made on this? Did a Jira get created? >>> >>> I have been debugging our backup scripts and seem to have found the same >>> problem. >>> >>> As far as I can work out so far, it seems that this happens when a new >>> snapshot is created and the old snapshot is being tarred. >>> >>> I get a similar message: >>> >>> /bin/tar: >>> var/lib/cassandra/backup/keyspacename/tablename-4eec3b01aba811e896342351775ccc66/snapshots/csbackup_2022-03-22T14\\:04\\:05/nb-523601-big-Data.db: >>> file changed as we read it >>> >>> Thanks >>> >>> Paul >>> >>> >>> >>> On 19 Mar 2022, at 02:41, Dinesh Joshi <djo...@apache.org> wrote: >>> >>> Do you have a repro that you can share with us? If so, please file a >>> jira and we'll take a look. >>> >>> On Mar 18, 2022, at 12:15 PM, James Brown <jbr...@easypost.com> wrote: >>> >>> This in 4.0.3 after running nodetool snapshot that we're seeing >>> sstables change, yes. >>> >>> James Brown >>> Infrastructure Architect @ easypost.com >>> >>> >>> On 2022-03-18 at 12:06:00, Jeff Jirsa <jji...@gmail.com> wrote: >>> >>>> This is nodetool snapshot yes? 3.11 or 4.0? >>>> >>>> In versions prior to 3.0, sstables would be written with -tmp- in the >>>> name, then renamed when complete, so an sstable definitely never changed >>>> once it had the final file name. With the new transaction log mechanism, we >>>> use one name and a transaction log to note what's in flight and what's not, >>>> so if the snapshot system is including sstables being written (from flush, >>>> from compaction, or from streaming), those aren't final and should be >>>> skipped. >>>> >>>> >>>> >>>> >>>> On Fri, Mar 18, 2022 at 11:46 AM James Brown <jbr...@easypost.com> >>>> wrote: >>>> >>>>> We use the boring combo of cassandra snapshots + tar to backup our >>>>> cassandra nodes; every once in a while, we'll notice tar failing with the >>>>> following: >>>>> >>>>> tar: >>>>> data/addresses/addresses-eb0196100b7d11ec852b1541747d640a/snapshots/backup20220318183708/nb-167-big-Data.db: >>>>> file changed as we read it >>>>> >>>>> I find this a bit perplexing; what would cause an sstable inside a >>>>> snapshot to change? The only thing I can think of is an incremental repair >>>>> changing the "repaired_at" flag on the sstable, but it seems like that >>>>> should "un-share" the hardlinked sstable rather than running the risk of >>>>> mutating a snapshot. >>>>> >>>>> >>>>> James Brown >>>>> Cassandra admin @ easypost.com >>>>> >>>> >>> >>>