I think this has a much simpler answer:  GNU tar interprets inode changes
as "changes" as well as block contents.  This includes the hardlink count.
I actually ended up working around it by using bsdtar, which doesn't
interpret hardlink count changes as a change to be concerned about.

On Tue, Mar 22, 2022 at 6:56 PM James Brown <jbr...@easypost.com> wrote:

> I filed https://issues.apache.org/jira/browse/CASSANDRA-17473 for this
> thread as a whole.
>
> Would you like a separate Jira issue on the matter of documenting how to
> tell when a snapshot is "ready"?
>
> James Brown
> Infrastructure Architect @ easypost.com
>
>
> On 2022-03-22 at 17:41:23, Dinesh Joshi <djo...@apache.org> wrote:
>
>> Cassandra creates hardlinks[1] first and then writes the manifest[2]. But
>> that is not the last thing it writes either[3]. This should definitely be
>> documented. Could you please open a jira?
>>
>> [1]
>> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1956
>> [2]
>> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1977
>> [3]
>> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L1981
>>
>> On Mar 22, 2022, at 4:53 PM, James Brown <jbr...@easypost.com> wrote:
>>
>>
>> There are not overlapping snapshots, so I don't think it's a second
>> snapshot. There are overlapping repairs.
>>
>>
>> > How does the backup process ensure the snapshot is taken before
>> starting to upload it ?
>>
>>
>> It just runs nice nodetool ${jmx_args[@]} snapshot -t "$TAG"
>> ${keyspaces[@]}
>>
>>
>> > A snapshot is only safe to use after the "manifest.json" file is
>> written.
>>
>>
>> Is this true? I don't see this anywhere in the documentation for
>> Cassandra (I would expect it on the Backups page, for example) or in the
>> help of nodetool snapshot. It was my understanding that when the nodetool
>> snapshot process finished, the snapshot was done. If that's wrong, it
>> definitely could be that we're just jumping the gun.
>>
>>
>> James Brown
>>
>> Infrastructure Architect @ easypost.com
>>
>>
>>
>> On 2022-03-22 at 10:38:56, Paul Chandler <p...@redshots.com> wrote:
>>
>> > Hi Yifan,
>>
>> >
>>
>> > It looks like you are right, I can reproduce this, when creating the
>> second snapshot the ctime does get updated to the time of the second
>> snapshot.
>>
>> >
>>
>> > I guess this is what is causing tar to produce the error.
>>
>> >
>>
>> > Paul
>>
>> >
>>
>> >> On 22 Mar 2022, at 17:12, Yifan Cai <yc25c...@gmail.com> wrote:
>>
>> >>
>>
>> >> 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
>>
>> >>>
>>
>> >>
>>
>> >
>>
>>
>>

-- 
This email, including its contents and any attachment(s), may contain 
confidential and/or proprietary information and is solely for the review 
and use of the intended recipient(s). If you have received this email in 
error, please notify the sender and permanently delete this email, its 
content, and any attachment(s).  Any disclosure, copying, or taking of any 
action in reliance on an email received in error is strictly prohibited.

Reply via email to