How does the backup process ensure the snapshot is taken before starting to upload it ? A snapshot is only safe to use after the "manifest.json" file is written.
I wonder if the snapshot is being compressed while the snapshot file is still being created. Em ter., 22 de mar. de 2022 às 14:17, Paul Chandler <p...@redshots.com> escreveu: > I will do a few more tests to see if I can pin point what is causing this, > then I will create a Jira ticket. > > This is actually a copy of a cluster that I am testing with, so the only > writes happening to the cluster are internal ones, so I will be surprised > if it is compaction or memtable flushes on the offending tables. There > could be repairs going on on the cluster through. > > This is a 4.0.0 cluster, but I think I have the same problem on a 3.11.6 > cluster, but not tested the 3.11.6 version yet. > > So I will try and get as much detail together before creating the ticket. > > Thanks > > Paul > > > On 22 Mar 2022, at 17:01, 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 >>>>> >>>> >>> >>> >