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
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 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.jav
> It was my understanding that when the nodetool snapshot process finished,
the snapshot was done.
This is correct. But snapshots could be partially available when using
incremental_backups or snapshot_before_compaction option.
If the compression/upload process starts after nodetool snapshot fini
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 o
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 wrote:
>
> I am wondering if the
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
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
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:
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 t
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 wrote:
> Hi all
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 ge
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 wrote:
>
> This in 4.0.3 after running nodetool snapshot that we're seeing sstables
> change, yes.
>
> James Brown
> Infrastructure Architect @ easypost
If that's the case, it sounds like a bug to me. A SSTable file in the
snapshot should never be modified by Cassandra, as that may interfere
with some tools backing up data from the snapshots.
On 18/03/2022 19:15, James Brown wrote:
This in 4.0.3 after running |nodetool snapshot| that we're seei
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 wrote:
> This is nodetool snapshot yes? 3.11 or 4.0?
>
> In versions prior to 3.0, sstables would be written with -tm
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
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 fi
17 matches
Mail list logo