[DISCUSS] Enhanced Disk Error Handling

2023-03-08 Thread Bowen Song via dev
At the moment, when a read error, such as unrecoverable bit error or 
data corruption, occurs in the SSTable data files, regardless of the 
disk_failure_policy configuration, manual (or to be precise, external) 
intervention is required to recover from the error.


Commonly, there's two approach to recover from such error:

1. The safer, but slower recover strategy: replace the entire node.
2. The less safe, but faster recover strategy: shut down the node,
   delete the affected SSTable file(s), and then bring the node back
   online and run repair.

Based on my understanding of Cassandra, it should be possible to recover 
from such error by marking the affected token range in the existing 
SSTable as "corrupted" and stop reading from them (e.g. creating a "bad 
block" file or in memory), and then streaming the affected token range 
from the healthy replicas. The corrupted SSTable file can then be 
removed upon the next successful compaction involving it, or 
alternatively an anti-compaction is performed on it to remove the 
corrupted data.


The advantage of this strategy is:

 * Reduced node down time - node restart or replacement is not needed
 * Less data streaming is required - only the affected token range
 * Faster recovery time - less streaming and delayed compaction or
   anti-compaction
 * No less safe than replacing the entire node
 * This process can be automated internally, removing the need for
   operator inputs

The disadvantage is added complexity on the SSTable read path and it may 
mask disk failures from the operator who is not paying attention to it.


What do you think about this?


Re: [DISCUSS] Enhanced Disk Error Handling

2023-03-08 Thread Bowen Song via dev

   /– A repair of the affected range would need to be completed among
   the replicas without such corruption (including paxos repair)./

It can be safe without a repair by over-streaming the data from more (or 
all) available replicas, either within the DC (when LOCAL_* CL is used) 
or across the whole cluster (when other CL is used), then perform a 
compaction locally on the streamed SSTables to get rid of the duplicate 
data. Since the read error should only affect a fairly limited range of 
tokens, over-streaming in theory should not be an issue.



   /– And we'd need a mechanism to execute repair on the affected node
   without it being available to respond to queries, either via the
   client protocol or via internode (similar to a partial bootstrap)./

The mechanism to not respond to queries already exists. I believe there 
may be better ways to do this, but at the minimal level, the affected 
node could just drop that read request silently, and then the 
coordinator will automatically retry it on other replicas if speculative 
retry is enabled, or the client may get a query failure (the "required 
responses N, received responses N-1" error).



   /My hunch is that the examples where this are desirable might be
   limited though. It might allow one to limp along on a bad drive
   momentarily while a proper replacement is bootstrapped, but
   typically with disk failures where there's smoke there's fire - I
   wouldn't expect a drive reporting uncorrectable errors / filesystem
   corruption to be long for this world./

Actually no. Regardless it's a mechanical hard drive or an SSD, they all 
have certain level of uncorrectable bit-error rate (UBER).


For example, a consumer grade hard drive may have an UBER of 1 in 1e14, 
that means on average roughly every 11 TiB read will lead to an 
unrecoverable read error, which result in an entire 512 bytes or 4096 
bytes sector becomes unreadable, and that's perfectly normal, the hard 
drive is still in good health and may still last for many years if not 
decades. Consumer grade SSDs often have UBER of 1 in 1e15, and data 
centre grade SSDs have far better UBER than consumer grade drives, but 
even then, the best still have UBER of about 1 in 1e17.


When managing a cluster of hundreds of Cassandra nodes, each has 
hundreds (if not thousands) GB of data read per day, the probability of 
hitting uncorrectable bit-error is pretty high. The Cassandra cluster of 
approximately 300 nodes I manage hits this fairly often, and replacing 
node for the sake of data consistency has become a chore.



On 08/03/2023 16:53, C. Scott Andreas wrote:

For this to be safe, my understanding is that:

– A repair of the affected range would need to be completed among the 
replicas without such corruption (including paxos repair).
– And we'd need a mechanism to execute repair on the affected node 
without it being available to respond to queries, either via the 
client protocol or via internode (similar to a partial bootstrap).


My hunch is that the examples where this are desirable might be 
limited though. It might allow one to limp along on a bad drive 
momentarily while a proper replacement is bootstrapped, but typically 
with disk failures where there's smoke there's fire - I wouldn't 
expect a drive reporting uncorrectable errors / filesystem corruption 
to be long for this world.


Can you say more about the scenarios you have in mind?

– Scott

On Mar 8, 2023, at 5:24 AM, Bowen Song via dev 
 wrote:



At the moment, when a read error, such as unrecoverable bit error or 
data corruption, occurs in the SSTable data files, regardless of the 
disk_failure_policy configuration, manual (or to be precise, 
external) intervention is required to recover from the error.


Commonly, there's two approach to recover from such error:

 1. The safer, but slower recover strategy: replace the entire node.
 2. The less safe, but faster recover strategy: shut down the node,
delete the affected SSTable file(s), and then bring the node back
online and run repair.

Based on my understanding of Cassandra, it should be possible to 
recover from such error by marking the affected token range in the 
existing SSTable as "corrupted" and stop reading from them (e.g. 
creating a "bad block" file or in memory), and then streaming the 
affected token range from the healthy replicas. The corrupted SSTable 
file can then be removed upon the next successful compaction 
involving it, or alternatively an anti-compaction is performed on it 
to remove the corrupted data.


The advantage of this strategy is:

  * Reduced node down time - node restart or replacement is not needed
  * Less data streaming is required - only the affected token range
  * Faster recovery time - less streaming and delayed compaction or
anti-compaction
  * No less safe than replacing the entire node
  * This process can be autom

Re: [DISCUSS] Enhanced Disk Error Handling

2023-03-09 Thread Bowen Song via dev

Hi Jeremiah,

I'm fully aware of that, which is why I said that deleting the affected 
SSTable files is "less safe".


If the "bad blocks" logic is implemented and the node abort the current 
read query when hitting a bad block, it should remain safe, as the data 
in other SSTable files will not be used. The streamed data should 
contain the unexpired tombstones, and that's enough to keep the data 
consistent on the node.


Cheers,
Bowen


On 09/03/2023 15:58, Jeremiah D Jordan wrote:
It is actually more complicated than just removing the sstable and 
running repair.


In the face of expired tombstones that might be covering data in other 
sstables the only safe way to deal with a bad sstable is wipe the 
token range in the bad sstable and rebuild/bootstrap that range (or 
wipe/rebuild the whole node which is usually the easier way).  If 
there are expired tombstones in play, it means they could have already 
been compacted away on the other replicas, but may not have compacted 
away on the current replica, meaning the data they cover could still 
be present in other sstables on this node.  Removing the sstable will 
mean resurrecting that data.  And pulling the range from other nodes 
does not help because they can have already compacted away the 
tombstone, so you won’t get it back.


Tl;DR you can’t just remove the one sstable you have to remove all 
data in the token range covered by the sstable (aka all data that 
sstable may have had a tombstone covering).  Then you can stream from 
the other nodes to get the data back.


-Jeremiah

On Mar 8, 2023, at 7:24 AM, Bowen Song via dev 
 wrote:


At the moment, when a read error, such as unrecoverable bit error or 
data corruption, occurs in the SSTable data files, regardless of the 
disk_failure_policy configuration, manual (or to be precise, 
external) intervention is required to recover from the error.


Commonly, there's two approach to recover from such error:

 1. The safer, but slower recover strategy: replace the entire node.
 2. The less safe, but faster recover strategy: shut down the node,
delete the affected SSTable file(s), and then bring the node back
online and run repair.

Based on my understanding of Cassandra, it should be possible to 
recover from such error by marking the affected token range in the 
existing SSTable as "corrupted" and stop reading from them (e.g. 
creating a "bad block" file or in memory), and then streaming the 
affected token range from the healthy replicas. The corrupted SSTable 
file can then be removed upon the next successful compaction 
involving it, or alternatively an anti-compaction is performed on it 
to remove the corrupted data.


The advantage of this strategy is:

  * Reduced node down time - node restart or replacement is not needed
  * Less data streaming is required - only the affected token range
  * Faster recovery time - less streaming and delayed compaction or
anti-compaction
  * No less safe than replacing the entire node
  * This process can be automated internally, removing the need for
operator inputs

The disadvantage is added complexity on the SSTable read path and it 
may mask disk failures from the operator who is not paying attention 
to it.


What do you think about this?



Re: [DISCUSS] Enhanced Disk Error Handling

2023-03-09 Thread Bowen Song via dev

   /When we attempt to rectify any bit-error by streaming data from
   peers, we implicitly take a lock on token ownership. A user needs to
   know that it is unsafe to change token ownership in a cluster that
   is currently in the process of repairing a corruption error on one
   of its instances' disks./

I'm not sure about this.

Based on my knowledge, streaming does not require a lock on the token 
ownership, if the node subsequently lost the ownership of the token 
range being streamed, it will just end up with some extra SSTable files 
containing useless data, and the files will get deleted when nodetool 
cleanup is run.


BTW, just pointing out the obvious, streaming is neither repairing nor 
bootstrapping. The latter two may require a lock on the token ownership.


On 09/03/2023 19:56, Abe Ratnofsky wrote:
I'm not seeing any reasons why CEP-21 would make this more difficult 
to implement, besides the fact that it hasn't landed yet.


There are two major potential pitfalls that CEP-21 would help us avoid:
1. Bit-errors beget further bit-errors, so we ought to be resistant to 
a high frequency of corruption events
2. Avoid token ownership changes when attempting to stream a corrupted 
token


I found some data supporting (1) - 
https://www.flashmemorysummit.com/English/Collaterals/Proceedings/2014/20140806_T1_Hetzler.pdf


If we detect bit-errors and store them in system_distributed, then we 
need a capacity to throttle that load and ensure that consistency is 
maintained.


When we attempt to rectify any bit-error by streaming data from peers, 
we implicitly take a lock on token ownership. A user needs to know 
that it is unsafe to change token ownership in a cluster that is 
currently in the process of repairing a corruption error on one of its 
instances' disks. CEP-21 makes this sequencing safe, and provides 
abstractions to better expose this information to operators.


--
Abe


On Mar 9, 2023, at 10:55 AM, Josh McKenzie  wrote:

Personally, I'd like to see the fix for this issue come after 
CEP-21. It could be feasible to implement a fix before then, that 
detects bit-errors on the read path and refuses to respond to the 
coordinator, implicitly having speculative execution handle the 
retry against another replica while repair of that range happens. 
But that feels suboptimal to me when a better framework is on the 
horizon.
I originally typed something in agreement with you but the more I 
think about this, the more a node-local "reject queries for specific 
token ranges" degradation profile seems like it _could_ work. I don't 
see an obvious way to remove the need for a human-in-the-loop on 
fixing things in a pre-CEP-21 world without opening pandora's box 
(Gossip + TMD + non-deterministic agreement on ownership state 
cluster-wide /cry).


And even in a post CEP-21 world you're definitely in the "at what 
point is it better to declare a host dead and replace it" fuzzy 
territory where there's no immediately correct answers.


A system_distributed table of corrupt token ranges that are currently 
being rejected by replicas with a mechanism to kick off a repair of 
those ranges could be interesting.


On Thu, Mar 9, 2023, at 1:45 PM, Abe Ratnofsky wrote:
Thanks for proposing this discussion Bowen. I see a few different 
issues here:


1. How do we safely handle corruption of a handful of tokens without 
taking an entire instance offline for re-bootstrap? This includes 
refusal to serve read requests for the corrupted token(s), and 
correct repair of the data.
2. How do we expose the corruption rate to operators, in a way that 
lets them decide whether a full disk replacement is worthwhile?
3. When CEP-21 lands it should become feasible to support ownership 
draining, which would let us migrate read traffic for a given token 
range away from an instance where that range is corrupted. Is it 
worth planning a fix for this issue before CEP-21 lands?


I'm also curious whether there's any existing literature on how 
different filesystems and storage media accommodate bit-errors 
(correctable and uncorrectable), so we can be consistent with those 
behaviors.


Personally, I'd like to see the fix for this issue come after 
CEP-21. It could be feasible to implement a fix before then, that 
detects bit-errors on the read path and refuses to respond to the 
coordinator, implicitly having speculative execution handle the 
retry against another replica while repair of that range happens. 
But that feels suboptimal to me when a better framework is on the 
horizon.


--
Abe

On Mar 9, 2023, at 8:23 AM, Bowen Song via dev 
 wrote:


Hi Jeremiah,

I'm fully aware of that, which is why I said that deleting the 
affected SSTable files is "less safe".


If the "bad blocks" logic is implemented and the node abort the 
current read query when hitting a bad block, it should remain safe, 
as the data i

Re: [DISCUSS] Enhanced Disk Error Handling

2023-03-09 Thread Bowen Song via dev
On Mar 9, 2023, at 10:55 AM, Josh McKenzie  
wrote:


Personally, I'd like to see the fix for this issue come after 
CEP-21. It could be feasible to implement a fix before then, that 
detects bit-errors on the read path and refuses to respond to the 
coordinator, implicitly having speculative execution handle the 
retry against another replica while repair of that range happens. 
But that feels suboptimal to me when a better framework is on the 
horizon.
I originally typed something in agreement with you but the more I 
think about this, the more a node-local "reject queries for 
specific token ranges" degradation profile seems like it _could_ 
work. I don't see an obvious way to remove the need for a 
human-in-the-loop on fixing things in a pre-CEP-21 world without 
opening pandora's box (Gossip + TMD + non-deterministic agreement 
on ownership state cluster-wide /cry).


And even in a post CEP-21 world you're definitely in the "at what 
point is it better to declare a host dead and replace it" fuzzy 
territory where there's no immediately correct answers.


A system_distributed table of corrupt token ranges that are 
currently being rejected by replicas with a mechanism to kick off a 
repair of those ranges could be interesting.


On Thu, Mar 9, 2023, at 1:45 PM, Abe Ratnofsky wrote:
Thanks for proposing this discussion Bowen. I see a few different 
issues here:


1. How do we safely handle corruption of a handful of tokens 
without taking an entire instance offline for re-bootstrap? This 
includes refusal to serve read requests for the corrupted 
token(s), and correct repair of the data.
2. How do we expose the corruption rate to operators, in a way 
that lets them decide whether a full disk replacement is worthwhile?
3. When CEP-21 lands it should become feasible to support 
ownership draining, which would let us migrate read traffic for a 
given token range away from an instance where that range is 
corrupted. Is it worth planning a fix for this issue before CEP-21 
lands?


I'm also curious whether there's any existing literature on how 
different filesystems and storage media accommodate bit-errors 
(correctable and uncorrectable), so we can be consistent with 
those behaviors.


Personally, I'd like to see the fix for this issue come after 
CEP-21. It could be feasible to implement a fix before then, that 
detects bit-errors on the read path and refuses to respond to the 
coordinator, implicitly having speculative execution handle the 
retry against another replica while repair of that range happens. 
But that feels suboptimal to me when a better framework is on the 
horizon.


--
Abe

On Mar 9, 2023, at 8:23 AM, Bowen Song via dev 
 wrote:


Hi Jeremiah,

I'm fully aware of that, which is why I said that deleting the 
affected SSTable files is "less safe".


If the "bad blocks" logic is implemented and the node abort the 
current read query when hitting a bad block, it should remain 
safe, as the data in other SSTable files will not be used. The 
streamed data should contain the unexpired tombstones, and that's 
enough to keep the data consistent on the node.



Cheers,
Bowen


On 09/03/2023 15:58, Jeremiah D Jordan wrote:
It is actually more complicated than just removing the sstable 
and running repair.


In the face of expired tombstones that might be covering data in 
other sstables the only safe way to deal with a bad sstable is 
wipe the token range in the bad sstable and rebuild/bootstrap 
that range (or wipe/rebuild the whole node which is usually the 
easier way).  If there are expired tombstones in play, it means 
they could have already been compacted away on the other 
replicas, but may not have compacted away on the current 
replica, meaning the data they cover could still be present in 
other sstables on this node.  Removing the sstable will mean 
resurrecting that data.  And pulling the range from other nodes 
does not help because they can have already compacted away the 
tombstone, so you won’t get it back.


Tl;DR you can’t just remove the one sstable you have to remove 
all data in the token range covered by the sstable (aka all data 
that sstable may have had a tombstone covering).  Then you can 
stream from the other nodes to get the data back.


-Jeremiah

On Mar 8, 2023, at 7:24 AM, Bowen Song via 
dev 
<mailto:dev@cassandra.apache.org>wrote:


At the moment, when a read error, such as unrecoverable bit 
error or data corruption, occurs in the SSTable data files, 
regardless of the disk_failure_policy configuration, manual (or 
to be precise, external) intervention is required to recover 
from the error.


Commonly, there's two approach to recover from such error:

 1. The safer, but slower recover strategy: replace the entire
node.
 2. The less safe, but faster recover strategy: shut down the
node, delete the affected SSTable file(s), and then bring
the node back

Re: [DISCUSS] Change the useage of nodetool tablehistograms

2023-03-16 Thread Bowen Song via dev

The documented command options are:

   nodetool tablehistograms [  | ]


That means one parameter will be treated as dot separated keyspace and 
table. Alternatively, two parameters will be treated as the keyspace and 
table respectively.


To remain compatible with the documented behaviour, my suggestion is to 
change the command options to:


   nodetool tablehistograms [  [ [...]] |
[ [...]]]

Feel free to add the "all except ..." feature to the above.

This doesn't break backward compatibility in documented ways. It only 
changes the undocumented behaviour. If someone is using the undocumented 
behaviour, they must know things may break when the software is 
upgraded. We can just add a line to the NEWS.txt and let them update 
their scripts.



On 16/03/2023 08:53, guo Maxwell wrote:

Hello everyone :
The nodetool tablehistograms have one argument which you can fill with 
only one table name with the format "keyspace_name.table_name 
/keyspace_name table_name", so that you can get the table histograms 
of the specied table.


And  if none arguments is set, all the tables' histograms will be 
print out.And if more than 2 arguments (nomatter the format is right 
or wrong) are set , all the tables' histograms will also be print out 
too(Which is a bug In my mind).


So the usage of nodetool tablehistograms has some usage restrictions, 
That is either output one , or all informations.


As CASSANDRA-18296 
 described , I 
will change the usage of nodetool tablehistograms, which support the 
feature below:
1. nodetool tablehistograms ks.tb1 ks.tb2  //print out list of 
tables' histograms with format keyspace.table
2.nodetool tablehistograms ks1 ks2 ks3 ... //print out list of 
keyspaces histograms
3.nodetool tablehistograms -i ks1 ks2  //print out list of table 
histograms except for the keyspaces list behind the option -i
4.nodetool tablehistograns -i ks ks.tb // print out list tables' 
histograms except for table in keyspace ks and ks.tb table.

5.none option specified ,then all tables histograms will be print out.

The usage will breaks compatibility with how it was done previously, 
and as this is a user facing tool.


So, What do you think?

Thanks~~~


Re: [DISCUSS] Change the useage of nodetool tablehistograms

2023-03-22 Thread Bowen Song via dev

TBH, the syntax looks unnecessarily complex and confusing to me.

For example, for this command:

   nodetool tablehistograns -ks ks1 -i -tbs ks1.tb1 ks2.tb2

Which one of the following should it do?

1. all tables in the keyspace ks1,  except the table tb1; or
2. all tables in all keyspaces, except any table in the keyspace ks1
   and the table tb2 in the keyspace ks2


I personally would prefer the simplicity of this approach:

   nodetool tablehistograms ks1 tb1 tb2 tb3

   nodetool tablehistograms ks1.tb1 ks1.tb2 ks2.tb3

   nodetool tablehistograms -i ks1 -i ks2

   nodetool tablehistograms -i ks1.tb1 -i ks2.tb2

They are self-explanatory. You don't need to read comments to understand 
what do they do, as long as you know that "-i" means "exclude".


A more complex and possibly confusing option could be:

   nodetool tablehistograms ks1 -i ks1.tb1 -i ks1.tb2  # all tables in
   the keyspace ks1, except the table tb1 and tb2

   nodetool tablehistograms -i ks1.tb1 -i ks1.tb2 ks1  # identical as
   above, as -i takes only one parameter

To avoid the above confusion, the command could enforce that the "-i" 
option may only be used after any positional options, thus makes the 2nd 
command a syntax error.


Beyond that, I don't see why the user can't make multiple invocations of 
the nodetool tablehistograms command if they have more complex or 
specific need.


For example, in this case:

   /> 6.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 -ks ks1 // print
   out list tables' histograms except for table in ks.tb1 ks.tb2 and
   all tables in ks1/

The same result can be achieved by concatenating the outputs of the 
following two commands:


   nodetool tablehistograms -i ks -i ks1

   nodetool tablehistograms ks -i ks.tb1 -i ks.tb2


On 22/03/2023 05:12, guo Maxwell wrote:
Thanks everyone , So It seems that it is better to add new parameter 
options to meet our needs, while keeping the original parameter 
functions unaffected to achieve backward compatibility.

So the new options are :
1. nodetool tablehistograms ks.tb1 or ks tb1  ... //this is *one of 
the old way *of using tablehistogram. will print out the histograms of 
tabke ks.tb1 , we keep the old format to print out the table 
histograms,besides if more than two arguments is provied, suchu as 
nodetool tablehistograms system.local system_schema.columns 
system_schema.tables then all tables's histograms will be printed out 
(I think this is a bug that not as excepted in the document's 
decription, we should remind the user that this is an incorrenct usage)


2. nodetool tablehistograms -tbs ks.tb1 ks.tb2  //print out list 
of tables' histograms with format keyspace.table
3.nodetool tablehistograms -ks ks1 ks2 ks3 ... //print out list of 
keyspaces histograms
4.nodetool tablehistograms -i -ks ks1 ks2  //print out list of 
table histograms except for the keyspaces list behind the option -i
5.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 // print out list 
tables' histograms except for table in ks.tb1 ks.tb2
6.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 -ks ks1 // print out 
list tables' histograms except for table in ks.tb1 ks.tb2 and all 
tables in ks1
6.none option specified ,then all tables histograms will be print 
out.// this is *another one of the old way* of using tablehistogram.


So we add some more options like "-i", "-ks", "-tbs" , we can combine 
these options  and we can also use any of them individually, besides, 
we can also use the tool through old way if a table with format ks.tb 
is provied.



Jeremiah D Jordan  于2023年3月16日周四 
23:14写道:


-1 on any change which breaks the previously documented usage.
+1 any additions to what the tool can do without breaking
previously documented behavior.


On Mar 16, 2023, at 7:42 AM, Josh McKenzie 
wrote:

We could also consider augmenting the tool with new named
arguments with the functionality you described and leave the
positional usage intact.

On Thu, Mar 16, 2023, at 6:43 AM, Bowen Song via dev wrote:


The documented command options are:

nodetool tablehistograms [  | ]



That means one parameter will be treated as dot separated
keyspace and table. Alternatively, two parameters will be
treated as the keyspace and table respectively.

To remain compatible with the documented behaviour, my
suggestion is to change the command options to:

nodetool tablehistograms [  [
[...]] |  [[...]]]

Feel free to add the "all except ..." feature to the above.

This doesn't break backward compatibility in documented ways. It
only changes the undocumented behaviour. If someone is using the
undocumented behaviour, they must know things may break when the
software is upgraded. We can just add a line to the NEWS.txt and
let them update their scripts.


On 

Re: [DISCUSS] Change the useage of nodetool tablehistograms

2023-03-23 Thread Bowen Song via dev
d options' behavior is same with nodetool tablestats , 
the difference is I displayed parameters specifying option -ks and 
-tbs , but tablestats don't.





Josh McKenzie  于2023年3月22日周三 23:35写道:

Agree w/Bowen. I think the straight forward simplicity of "clear
inclusion and exclusion semantics, default to include all in scope
    excepting things that are explicitly ignored" would be ideal.


On Wed, Mar 22, 2023, at 8:45 AM, Bowen Song via dev wrote:


TBH, the syntax looks unnecessarily complex and confusing to me.

For example, for this command:

nodetool tablehistograns -ks ks1 -i -tbs ks1.tb1 ks2.tb2

Which one of the following should it do?

 1. all tables in the keyspace ks1,  except the table tb1; or
 2. all tables in all keyspaces, except any table in the keyspace
ks1 and the table tb2 in the keyspace ks2


I personally would prefer the simplicity of this approach:

nodetool tablehistograms ks1 tb1 tb2 tb3

nodetool tablehistograms ks1.tb1 ks1.tb2 ks2.tb3

nodetool tablehistograms -i ks1 -i ks2

nodetool tablehistograms -i ks1.tb1 -i ks2.tb2


They are self-explanatory. You don't need to read comments to
understand what do they do, as long as you know that "-i" means
"exclude".

A more complex and possibly confusing option could be:



nodetool tablehistograms ks1 -i ks1.tb1 -i ks1.tb2  # all
tables in the keyspace ks1, except the table tb1 and tb2

nodetool tablehistograms -i ks1.tb1 -i ks1.tb2 ks1  #
identical as above, as -i takes only one parameter

To avoid the above confusion, the command could enforce that the
"-i" option may only be used after any positional options, thus
makes the 2nd command a syntax error.


Beyond that, I don't see why the user can't make multiple
invocations of the nodetool tablehistograms command if they have
more complex or specific need.

For example, in this case:

/> 6.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 -ks ks1
// print out list tables' histograms except for table in
ks.tb1 ks.tb2 and all tables in ks1/

The same result can be achieved by concatenating the outputs of
the following two commands:

nodetool tablehistograms -i ks -i ks1

nodetool tablehistograms ks -i ks.tb1 -i ks.tb2


On 22/03/2023 05:12, guo Maxwell wrote:

Thanks everyone , So It seems that it is better to add new
parameter options to meet our needs, while keeping the original
parameter functions unaffected to achieve backward compatibility.
So the new options are :
1. nodetool tablehistograms ks.tb1 or ks tb1  ... //this is *one
of the old way *of using tablehistogram. will print out the
histograms of tabke ks.tb1 , we keep the old format to print out
the table histograms,besides if more than two arguments is
provied, suchu as nodetool tablehistograms system.local
system_schema.columns system_schema.tables then all tables's
histograms will be printed out (I think this is a bug that not
as excepted in the document's decription, we should remind the
user that this is an incorrenct usage)

2. nodetool tablehistograms -tbs ks.tb1 ks.tb2  //print out
list of tables' histograms with format keyspace.table
3.nodetool tablehistograms -ks ks1 ks2 ks3 ... //print out list
of keyspaces histograms
4.nodetool tablehistograms -i -ks ks1 ks2  //print out list
of table histograms except for the keyspaces list behind the
option -i
5.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 // print out
list tables' histograms except for table in ks.tb1 ks.tb2
6.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 -ks ks1 //
print out list tables' histograms except for table in ks.tb1
ks.tb2 and all tables in ks1
6.none option specified ,then all tables histograms will be
print out.// this is *another one of the old way* of using
tablehistogram.

So we add some more options like "-i", "-ks", "-tbs" , we can
combine these options  and we can also use any of them
individually, besides, we can also use the tool through old way
if a table with format ks.tb is provied.


Jeremiah D Jordan  于2023年3月16日周四
23:14写道:

-1 on any change which breaks the previously documented usage.
+1 any additions to what the tool can do without breaking
previously documented behavior.


On Mar 16, 2023, at 7:42 AM, Josh McKenzie
 wrote:

        We could also consider augmenting the tool with new named
arguments with the functionality you described and leave
the positional usage intact.

On Thu, Mar 16, 2023, at 6:43 AM, Bowen Song via dev wrote:


The documented command options are:

nodetool 

Re: [DISCUSS] Change the useage of nodetool tablehistograms

2023-03-23 Thread Bowen Song via dev
In that case, I would recommend fix the bug that prints everything when 
an arbitrary number of arguments is given.


On 23/03/2023 13:40, guo Maxwell wrote:
firstly I think anything existing must be reasonable,so ignore option 
for tablestats must be a need for the user to use. at least I used it 
some time ;
secondly in order  to keep this as simple as possible ,I think left 
the option unchanged is enough ,because the original usage is simple 
enough. user can just print the specified table after set nodetool 
tablehistorgrams ks table ,and if there is ten tables in kesypace  ,it 
is simple for him to type ten times with different table names which I 
think at first Only set with argument ks keyspace name is enough.
When we just want to see eight tables in the ks ,the user should just 
type eight table name which ignore two table may be enough.





Bowen Song via dev 于2023年3月23日 
周四下午8:07写道:


I don't think the nodetool tablestats command's parameters should
be used as a reference implementation for the nodetool
tablehistograms command. Because:

  * the tablehistograms command can take the keyspace and table as
two separate parameters, but the tablestats command can't.
  * the tablestats command can take keyspace (without table) as a
parameter, but the tablehistograms command can't.

The introduction of the -ks and -tbs options are unnecessary for
the tablestats command, because it's parameters are:

nodetool tablestats [|
[|[...]]]

Which means any positional parameter without a dot is treated as a
keyspace name, otherwise it's treated as dot-separated keyspace
and table name. That, however, does not apply to the nodetool
tablehistograms command, which led to your workaround - the
addition of the -ks and -tbs options.

But if you could just forget about the nodetool tablestats command
for a moment, and look at the nodetool tablehistograms command
alone, you will see that it's unnecessary to introduce the -ks and
-tbs options, because the command already takes keyspace name and
table name, just in a different format.

In addition to that, I would be interested to know how often do
people use the -i option in the nodetool tablestats command. My
best guess is, very very rarely.

If my guess is correct, we should keep the nodetool
tablehistograms command as simple as:

nodetool tablehistograms [  [ [...]] |
 [[...]]]

It's good enough if the above can cover the majority of use cases.
The remaining use cases can be dealt with individually, by
multiple invocations of the same command or providing it with a
script-generated list of tables in the  format.

TL;DR: The KISS principle
<https://en.wikipedia.org/wiki/KISS_principle> should apply here -
keep it simple.


On 23/03/2023 03:05, guo Maxwell wrote:


Maybe I didn't describe the usage of option "-i" clearly, The
reason why I think the command argument should be like this :


1. nodetool tablehistograms ks.tb1 or ks tb1  ... //this is
*one of the old way *of using tablehistogram. will print out
the histograms of tabke ks.tb1 , we keep the old format to
print out the table histograms,besides if more than two
arguments is provied, suchu as nodetool tablehistograms
system.local system_schema.columns system_schema.tables then
all tables's histograms will be printed out (I think this is
a bug that not as excepted in the document's decription, we
should remind the user that this is an incorrenct usage)

2. nodetool tablehistograms -tbs ks.tb1 ks.tb2  //print
out list of tables' histograms with format keyspace.table
3.nodetool tablehistograms -ks ks1 ks2 ks3 ... //print out
list of keyspaces histograms
4.nodetool tablehistograms -i -ks ks1 ks2  //print out
list of table histograms except for the keyspaces list behind
the option -i
5.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 // print out
list tables' histograms except for table in ks.tb1 ks.tb2
6.nodetool tablehistograns -i -tbs ks.tb1 ks.tb2 -ks ks1 //
print out list tables' histograms except for table in ks.tb1
ks.tb2 and all tables in ks1
6.none option specified ,then all tables histograms will be
print out.// this is *another one of the old way* of using
tablehistogram.


 is to make the command format  to be consistent with the format
of nodetool tablestats, so for users, there will be a unified
awareness of using these two commands, rather than different
commands requiring different usage awareness , we can see the
description of the tablestats doc for option "-i "

Ignore the list of tables and display the re

Re: [DISCUSS] Introduce DATABASE as an alternative to KEYSPACE

2023-04-04 Thread Bowen Song via dev
I personally prefer to use the name "keyspace", because it avoids the 
confusion between the "database software/server" and the "collection of 
tables in a database". "An SQL database" can mean different things in 
different contexts, but "a Cassandra keyspace" always mean the same thing.


On 04/04/2023 16:48, Mike Adamson wrote:

Hi,

I'd like to propose that we add DATABASE to the CQL grammar as an 
alternative to KEYSPACE.


Background: While TABLE was introduced as an alternative for 
COLUMNFAMILY in the grammar we have kept KEYSPACE for the container 
name for a group of tables. Nearly all traditional SQL databases use 
DATABASE as the container name for a group of tables so it would make 
sense for Cassandra to adopt this naming as well.


KEYSPACE would be kept in the grammar but we would update some logging 
and documentation to encourage use of the new name.


Mike Adamson

--
DataStax Logo Square   *Mike Adamson*
Engineering

+1 650 389 6000 |datastax.com 



Find DataStax Online: 	LinkedIn Logo 
 
Facebook Logo 
 
Twitter Logo  RSS Feed 
 Github Logo 



Re: [DISCUSS] Introduce DATABASE as an alternative to KEYSPACE

2023-04-06 Thread Bowen Song via dev

/> I'm quite happy to leave things as they are if that is the consensus./

+1 to the above


On 06/04/2023 14:54, Mike Adamson wrote:
My apologies. I started this discussion off the back of a usability 
discussion around new user accessibility to Cassandra and the premise 
that there is an initial steep learning curve for new users. Including 
new users who have worked for a long time in the traditional DBMS field.


On the basis of the reason for the discussion,  TABLEGROUP doesn't sit 
well because of user types / functions / indexes etc. which are not 
strictly tables and is also yet another Cassandra only term.


NAMESPACE could work but it's different usage in other systems could 
be just as confusing to new users.


And, I certainly don't think having multiple names for the same thing 
just to satisfy different parties is a good idea at all.


I'm quite happy to leave things as they are if that is the consensus.

On Thu, 6 Apr 2023 at 14:16, Josh McKenzie  wrote:


KEYSPACE is fine. If we want to introduce a standard nomenclature
like DATABASE that’s also fine. Inventing brand new ones is not
fine, there’s no benefit.

I'm with Benedict in principle, with Aleksey in practice; I think
KEYSPACE and SCHEMA are actually fine enough.

If and when we get to any kind of multi-tenancy, having a more
metaphorical abstraction that users are familiar with like these
becomes more valuable; it's pretty clear that things in different
keyspaces, different databases, or even different schemas could
have different access rules, resourcing, etc from one another.

While the off-the-cuff logical TABLEGROUP thing is a /literal/
statement about what the thing is, it'd be another unique term to
us;  we have enough things in our system where we've charted our
own path. My personal .02 is we don't need to go adding more. :)

On Thu, Apr 6, 2023, at 8:54 AM, Mick Semb Wever wrote:


… but that should be a different discussion about how we
evolve config.



I disagree. Nomenclature being difficult can benefit from
holistic and forward thinking.
Sure you can label this off-topic if you like, but I value our
discuss threads being collaborative in an open-mode.
Sometimes the best idea is on the tail end of a sequence of bad
and/or unpopular ideas.








--
DataStax Logo Square   *Mike Adamson*
Engineering

+1 650 389 6000 |datastax.com 



Find DataStax Online: 	LinkedIn Logo 
 
Facebook Logo 
 
Twitter Logo  RSS Feed 
 Github Logo 



Re: [DISCUSS] Maintain backwards compatibility after dependency upgrade in the 5.0

2023-06-28 Thread Bowen Song via dev
IMHO, anyone upgrading software between major versions should expect to 
see breaking changes. Introducing breaking or major changes is the whole 
point of bumping major version numbers.


Since the library upgrade need to happen sooner or later, I don't see 
any reason why it should not happen in the 5.0 release.



On 27/06/2023 19:21, Maxim Muzafarov wrote:

Hello everyone,


We use the Dropwizard Metrics 3.1.5 library, which provides a basic
set of classes to easily expose Cassandra internals to a user through
various interfaces (the most common being JMX). We want to upgrade
this library version in the next major release 5.0 up to the latest
stable 4.2.19 for the following reasons:
- the 3.x (and 4.0.x) Dropwizard Metrics library is no longer
supported, which means that if we face a critical CVE, we'll still
need to upgrade, so it's better to do it sooner and more calmly;
- as of 4.2.5 the library supports jdk11, jdk17, so we will be in-sync
[1] as well as having some of the compatibility fixes mentioned in the
related JIRA [2];
- there have been a few user-related requests [3][4] whose
applications collide with the old version of the library, we want to
help them;


The problem

The problem with simply upgrading is that the JmxReporter class of the
library has moved from the com.codahale.metrics package in the 3.x
release to the com.codahale.metrics.jmx package in the 4.x release.
This is a problem for applications/tools that rely on the cassandra
classpath (lib/jars) as after the upgrade they may be looking for the
JmxReporter class which has changed its location.

A good example of the problem that we (or a user) may face after the
upgrade is our tests and the cassandra-driver-core 3.1.1, which uses
the old 3.x version of the library in tests. Of course, in this case,
we can upgrade the cassandra driver up to 4.x [5][6] to fix the
problem, as the new driver uses a newer version of the library, but
that's another story I won't go into for now. I'm talking more about
visualising the problem a user might face after upgrading to 5.0 if
he/she rely on the cassandra classpath, but on the other hand, they
might not face this problem at all because, as I understand, they will
provide this library in their applications by themselves.


So, since Cassandra has a huge ecosystem and a variety of tools that I
can't even imagine, the main question here is:

Can we move forward with this change without breaking backwards
compatibility with any kind of tools that we have considering the
example above as the main case? Do you have any thoughts on this?

The changes are here:
https://github.com/apache/cassandra/pull/2238/files



[1] 
https://github.com/dropwizard/metrics/pull/2180/files#diff-5dbf1a803ecc13ff945a08ed3eb09149a83615e83f15320550af8e3a91976446R14
[2] https://issues.apache.org/jira/browse/CASSANDRA-14667
[3] https://github.com/dropwizard/metrics/issues/1581#issuecomment-628430870
[4] https://issues.apache.org/jira/browse/STORM-3204
[5] https://issues.apache.org/jira/browse/CASSANDRA-15750
[6] https://issues.apache.org/jira/browse/CASSANDRA-17231


Re: [DISCUSS] Add subscription mangement instructions to user@, dev@ message footers

2024-01-22 Thread Bowen Song via dev
Adding a footer or modifying the email content in any way will break the 
DKIM signature of the email if it has one. Since the mailing list's mail 
server will forward the emails to the recipients, the SPF check will 
fail too. Failing the DKIM signature & SPF check will result in the 
email likely being treated as spam and either end up in the spam/junk 
mailbox or being rejected by recipients' mail server. The DMARC standard 
also requires at least one of the DKIM signature and SPF check must 
pass, otherwise it is considered as a failure. If the sender domain has 
a valid DMARC rule to reject or quarantine the failing emails, the 
mailing list subscribers with a mail service provider supporting the 
DMARC standard will never see any email from these senders via the 
mailing list landing in their inbox.


Balancing the pros and cons, I believe it's better to have small number 
of users occasionally spamming the mailing lists with invalid 
unsubscription emails than having the vast majority of users unable to 
receive emails from a subset of users (e.g. anyone from the @yahoo.com 
domain, or myself).


On 22/01/2024 18:10, C. Scott Andreas wrote:

Hi all,

I'd like to propose appending the following two footers to messages 
sent to the user@ and dev@ lists. The proposed postscript including 
line breaks is between the "X" blocks below.


User List Footer:
X

---
Unsubscribe: Send a blank email to 
user-unsubscr...@cassandra.apache.org. Do not reply to this message.
Cassandra Community: Follow other mailing lists or join us in Slack: 
https://cassandra.apache.org/_/community.html

X

Dev List Footer:
X

---
Unsubscribe: Send a blank email to 
dev-unsubscr...@cassandra.apache.org. Do not reply to this message.
Cassandra Community: Follow other mailing lists or join us in Slack: 
https://cassandra.apache.org/_/community.html

X

Offering this proposal for three reasons:
– Many users are sending "Unsubscribe" messages to the full mailing 
list which prompts others to wish to unsubscribe – a negative cascade 
that affects the size of our user community.
– Many users don't know where to go to figure out how to unsubscribe, 
especially if they'd joined many years ago.
– Nearly all mailing lists provide a one-click mechanism for 
unsubscribing or built-in mail client integration to do so via message 
headers. Including compact instructions on how to leave is valuable to 
subscribers.


#asfinfra indicates that such footers can be appended given project 
consensus and an INFRA- ticket: 
https://the-asf.slack.com/archives/CBX4TSBQ8/p1705939868631079


If we reach consensus on adding a message footer, I'll file an INFRA 
ticket with a link to this thread.


Thanks,

– Scott



Re: [DISCUSS] Add subscription mangement instructions to user@, dev@ message footers

2024-01-22 Thread Bowen Song via dev
Google Group works slightly differently. They "forward" emails using the 
group's email address as the "From" address, not the original sender's 
email address, unless the sender address happen to be a Google mail 
address (including Gmail and others). Technically speaking, that's not 
forwarding, but sending a new email with the original email's content, 
subject, sender name (but not address), etc. information copied over.


I believe the mailing list software this mailing list is using also 
supports such feature. For example, this email's "From" address is 
"Bowen Song via dev ", not my actual email 
address (which is the "Cc" address). If we add the footer to all emails, 
all "From" addresses, other than those Apache email addresses (e.g. 
f...@apache.org), will have to be turned into "dev@cassandra.apache.org". 
This works, but there's a catch. Many people (habitually) hits the 
"reply all" button on their mail client instead of the "reply" button, 
and as a result of that, the person being replied to will receive two 
nearly identical emails, one addressed to the mailing list which is then 
modified to added the footer, and the other Cc-ed to them without the 
footer. This may turn out to be very annoying if a mailing list 
participant can't (or doesn't know how to) setup inbox rules to filter 
these out.


There's no "Prefect Solution™", unsurprisingly.


On 22/01/2024 19:08, C. Scott Andreas wrote:

Bowen and Jeremiah, thanks for remembering this.

I'd remembered the DKIM/SPF issue, but not its relationship to the 
message footer - appreciate your work fixing that, Bowen.


I'm part of a few Google Groups that relay messages with an appended 
footer that don't seem to encounter invalidation, but am not curious 
enough to learn how they make that work right now. :)


I withdraw the proposal. 👍

– Scott


On Jan 22, 2024, at 10:56 AM, Brandon Williams  wrote:


That's right, I'd forgotten about this.  I change my +1 to -1, 
there's not enough value in this to break signatures.


Kind Regards,
Brandon


On Mon, Jan 22, 2024 at 12:42 PM Jeremiah Jordan 
 wrote:



Here was the thread where it was removed:
lists.apache.org
<https://lists.apache.org/thread/9wtw9m4r858xdm78krf1z74q3krc27st>
favicon.ico
<https://lists.apache.org/thread/9wtw9m4r858xdm78krf1z74q3krc27st>



On Jan 22, 2024, at 12:37 PM, J. D. Jordan
 wrote:
I think we used to have this and removed them because it was
breaking the encryption signature on messages or something which
meant they were very likely to be treated as spam?

Not saying we can’t put it back on, but it was removed for good
reasons from what I recall.


On Jan 22, 2024, at 12:19 PM, Brandon Williams
 wrote:

+1

Kind Regards,
Brandon


On Mon, Jan 22, 2024 at 12:10 PM C. Scott Andreas
 wrote:

Hi all,

I'd like to propose appending the following two footers to
messages sent to the user@ and dev@ lists. The proposed
postscript including line breaks is between the "X" blocks
below.

User List Footer:
X

---
Unsubscribe: Send a blank email to
user-unsubscr...@cassandra.apache.org. Do not reply to this
message.
Cassandra Community: Follow other mailing lists or join us in
Slack: https://cassandra.apache.org/_/community.html
X

Dev List Footer:
X

---
Unsubscribe: Send a blank email to
dev-unsubscr...@cassandra.apache.org. Do not reply to this
message.
Cassandra Community: Follow other mailing lists or join us in
Slack: https://cassandra.apache.org/_/community.html
X

Offering this proposal for three reasons:
– Many users are sending "Unsubscribe" messages to the full
mailing list which prompts others to wish to unsubscribe – a
negative cascade that affects the size of our user community.
– Many users don't know where to go to figure out how to
unsubscribe, especially if they'd joined many years ago.
– Nearly all mailing lists provide a one-click mechanism for
unsubscribing or built-in mail client integration to do so via
message headers. Including compact instructions on how to
leave is valuable to subscribers.

#asfinfra indicates that such footers can be appended given
project consensus and an INFRA- ticket:
https://the-asf.slack.com/archives/CBX4TSBQ8/p1705939868631079

If we reach consensus on adding a message footer, I'll file an
INFRA ticket with a link to this thread.

Thanks,

– Scott







Re: Table name length limit in Cassandra

2024-02-22 Thread Bowen Song via dev

Hi Gaurav,

I would be less worried about performance issues than interoperability 
issues. Other tools/client libraries do not expect this, and may cause 
them to behave unexpectedly (e.g. truncating/crashing/...).


If you can, try get rid of common prefix/suffix, and use abbreviations 
where possible. You shouldn't have thousands of tables (and yes, there's 
performance issue with that), so the table name length limit really 
shouldn't be an issue.


Best,
Bowen

On 22/02/2024 05:47, Gaurav Agarwal wrote:

Hi team,

Currently Cassandra has a table name length limit of 48 characters. If 
I understand correctly, it was made due to the fact that filename can 
not be more than 255 characters in windows. However, Linux supports up 
to 4096 bytes of file name.


Given my Cassandra nodes are on Linux systems, can I increase the 
limit from 48 characters to 64 characters? Will there be any 
performance issues due to increasing the limit?


Thanks
Gaurav


Re: [DISCUSS] New CQL command/option for listing roles with superuser privileges

2024-02-29 Thread Bowen Song via dev

I believe that opens the door to this kind of situations:

1. create superuser role "role1"
2. create superuser role "role2"
3. add "role2" to members of "role1"
4. remove "role2" from the members of "role1"
5. "role2" now inexplicitly lost the superuser state

TBH, my preferred solution is making superuser roles not inheritable. If 
a role has members, it cannot be made superuser; and if a role is 
superuser, no members can be added to it.


It doesn't make much sense to inherit from a superuser role, because it 
has unrestricted permissions, which renders any permission explicitly 
set on the child roles useless. This enforces the role to be made 
superuser explicitly, which makes all the display or filtering issues 
related to the inheritance goes away.


On 29/02/2024 11:30, Štefan Miklošovič wrote:
Why don't we just update the is_superuser column of a role when it 
effectively achieves a superuser status when it is granted some 
superuser role? Similarly, we would remove its superuser status when 
there are no superuser roles granted to it anymore.


I think that at least for the second case (when a superuser role is 
revoked and there is none anymore), there would need to be some 
transaction because as it checks if there are any superuser roles or 
not to set it to false, somebody else might grant that superuser role 
to it again so we might end up with having is_superuser set to false 
while it might still have a superuser role granted.


I am not sure if this is achievable and I am sorry if this was already 
answered / rejected elsewhere.


On Thu, Feb 29, 2024 at 11:33 AM  wrote:

Hi Maxwell,

Currently system_auth.roles table doesn’t have acquired superuser
info available in columns to filter on it. Below is the
system_auth.roles table for the example I have listed in the
previous email. If you notice, though role1 and role11 acquired
superuser status through grants, is_superuser column is False for
these roles and acquired superuser status is not apparent directly
from the columns of this table. member_of column shows immediate
parent/grant of a given role. But these grants can result in a
huge tree of roles hierarchy and there may be a role anywhere up
in the hierarchy which is a superuser.

cassandra@cqlsh> select * from system_auth.roles;

 role      | can_login | is_superuser | member_of  | salted_hash

---+---+--++--
     role2 |     False |    False |       null |                  
        null
    role11 |     False |    False |  {'role1'} |                  
        null
    super1 |     False |     True |       null |                  
        null
     role1 |     False |    False | {'super1'} |                  
        null


Thanks,
Shailaja



On Feb 29, 2024, at 2:11 AM, guo Maxwell 
wrote:

Hi ,
 1. can this cql "SELECT role from system_auth.roles where
is_superuser = True ALLOW FILTERING ;"  meet your needs if the
user to execute the cql have the right to do so  ?
 2. I think may be we can also add the ability to filter on list
role/user grammar, for example : list user where super = True;



Shailaja Koppu  于2024年2月28日周三 20:40写道:

Hi Team,

Currently LIST ROLES command doesn’t indicate if a role has
superuser privilege, if acquired through a grant in roles
hierarchy (LIST ROLES has super column true only if the role
is created with SUPERUSER=true). For example, in the below
example, super1 is a superuser, role1 acquired superuser
status through grant of super1 and role11 acquired superuser
status through grant of role1. LIST ROLES output has super
column true only for super1.


cassandra@cqlsh> create role super1 WITH SUPERUSER = true;
cassandra@cqlsh> create role role1;
cassandra@cqlsh> create role role11;
cassandra@cqlsh> create role role2;
cassandra@cqlsh> grant super1 to role1;
cassandra@cqlsh> grant role1 to role11;
cassandra@cqlsh> list roles;

 role      | super | login | options | datacenters
---+---+---+-+-
     role1 | False | False |        {} |         ALL
    role11 | False | False |        {} |         ALL
     role2 | False | False |        {} |         ALL
    super1 |  True | False |        {} |         ALL


One way to check has a role acquired superuser status is by
running LIST ROLES of  and looking for at least one
row with super column true. This works fine to check
superuser status of a given role.

cassandra@cqlsh> list roles of role11;

 role   | super | login | options | datacenters
+---+---+-+-
  role1 | False | Fal

Re: [DISCUSS] What SHOULD we do when we index an inet type that is ipv4?

2024-03-06 Thread Bowen Song via dev
Technically, 127.0.0.1 (IPv4) is not 0:0:0:0:0::7f00:0001 (IPv6), 
but their values are equal. Just like 1.0 (double) is not 1 (int), but 
their values are equal. So, what is the meaning of "=" in CQL?


On 06/03/2024 21:36, David Capwell wrote:

So, was reviewing SAI and found we convert ipv4 to ipv6 (which is valid for the 
type) and made me wonder what the behavior would be if client mixed ipv4 with 
ipv4 encoded as ipv6… this caused me to find a different behavior in SAI to the 
rest of C*… where I feel C* is doing the wrong thing…

Lets walk over a simple example

ipv4: 127.0.0.1
ipv6: 0:0:0:0:0::7f00:0001

Both of these address are equal according to networking and java… but for C* 
they are different!  These are 2 different values as ipv4 is 4 bytes and ipv6 
is 16 bytes, so 4 != 16!

With SAI we convert all ipv4 to ipv6 so that the search logic is correct… this 
causes SAI to return partitions that ALLOW FILTERING and other indexes wouldn’t…

This gets to the question in the subject… what SHOULD we do for this type?

I see 3 options:

1) SAI use the custom C* semantics where 4 != 16… this keeps us consistent…
2) ALLOW FILTERING and other indexes are “fixed” so that we actually match 
correctly… we are not really able to fix if the type is in a partition or 
clustering column though…
3) deprecate inet in favor of a inet_better type… where inet semantics is the 
custom C* semantics and inet_better handles this case

Thoughts?


Re: [DISCUSS] What SHOULD we do when we index an inet type that is ipv4?

2024-03-07 Thread Bowen Song via dev
I think the answer to that is, if an inet type column is a partition 
key, can I write to it in IPv4 and then query it with IPv6 and find the 
record? I believe the behaviour between SAI and partition key should be 
the same.


On 07/03/2024 17:43, Caleb Rackliffe wrote:
Yeah, what we have with inet is much like if we had a type like 
"numeric" that allowed you to write both ints and doubles. If we had 
actual "inet4" and "inet6" types, SAI would have been able to index 
them as fixed length values without doing the 4 -> 16 byte conversion. 
Given SAI could easily change this to go one way or another at 
post-filtering time, perhaps there's another option:


4.) Have an option on the column index that allows the user to specify 
whether ipv4 and ipv6 addresses are comparable. If they are, nothing 
changes. If they aren't, we can just take the matches from the index 
and filter "strictly".


I'm not sure what's best here, because what it seems to hinge on is 
what users actually want to do when they throw both v4 and v6 
addresses into a single column. Without any real loss in storage 
efficiency, you could index them in two separate columns on the same 
table, and none of this matters. If they are mixed, it feels like we 
should at least have the option to make them comparable, kind of like 
we have the option to make text case-insensitive or unicode normalized 
right now.


On Wed, Mar 6, 2024 at 4:35 PM Bowen Song via dev 
 wrote:


Technically, 127.0.0.1 (IPv4) is not 0:0:0:0:0::7f00:0001 (IPv6),
but their values are equal. Just like 1.0 (double) is not 1 (int),
but
their values are equal. So, what is the meaning of "=" in CQL?

On 06/03/2024 21:36, David Capwell wrote:
> So, was reviewing SAI and found we convert ipv4 to ipv6 (which
is valid for the type) and made me wonder what the behavior would
be if client mixed ipv4 with ipv4 encoded as ipv6… this caused me
to find a different behavior in SAI to the rest of C*… where I
feel C* is doing the wrong thing…
>
> Lets walk over a simple example
>
> ipv4: 127.0.0.1
> ipv6: 0:0:0:0:0::7f00:0001
>
> Both of these address are equal according to networking and
java… but for C* they are different!  These are 2 different values
as ipv4 is 4 bytes and ipv6 is 16 bytes, so 4 != 16!
>
> With SAI we convert all ipv4 to ipv6 so that the search logic is
correct… this causes SAI to return partitions that ALLOW FILTERING
and other indexes wouldn’t…
>
> This gets to the question in the subject… what SHOULD we do for
this type?
>
> I see 3 options:
>
> 1) SAI use the custom C* semantics where 4 != 16… this keeps us
consistent…
> 2) ALLOW FILTERING and other indexes are “fixed” so that we
actually match correctly… we are not really able to fix if the
type is in a partition or clustering column though…
> 3) deprecate inet in favor of a inet_better type… where inet
semantics is the custom C* semantics and inet_better handles this case
>
> Thoughts?


Re: Default table compression defined in yaml.

2024-03-19 Thread Bowen Song via dev
I believe the `foobar_in_kb: 123` format in the cassandra.yaml file is 
deprecated, and the new format is `foobar: 123KiB`. Is there a need to 
introduce new settings entries with the deprecated format only to be 
removed at a later version?



On 18/03/2024 14:39, Claude Warren, Jr via dev wrote:
After much work by several people, I have pulled together the changes 
to define the default compression in the cassandra.yaml file and have 
created a pull request [1].


If you are interested this in topic, please take a look at the changes 
and give at least a cursory review.


[1] https://github.com/apache/cassandra/pull/3168 



Thanks,
Claude

Re: Schema Disagreement Issue for Cassandra 4.1

2024-04-01 Thread Bowen Song via dev

It sounds worthy of a Jira ticket.

On 01/04/2024 06:23, Cheng Wang via dev wrote:

Hello,

I have recently encountered a problem concerning schema disagreement 
in Cassandra 4.1. It appears that the schema versions do not reconcile 
as expected.


The issue can be reproduced by following these steps:
- Disable the gossip in Node A.
- Make a schema change in Node B, such as creating a new table.
- Re-enable the gossip in Node A.

My expectation was that the schema versions would eventually 
reconcile. However, in Cassandra 4.1, it seems that reconciliation 
hangs indefinitely unless I reboot the node. Interestingly, when 
performing the same steps in Cassandra 3.0, the schema version 
synchronizes within about a minute.


Has anyone else experienced this issue with Cassandra 4.x? It appears 
to me that this could be a regression in the 4.x series.


Any insights or suggestions would be greatly appreciated.

Thanks,
Cheng


Re: [DISCUSS] Adding support for BETWEEN operator

2024-05-14 Thread Bowen Song via dev

Ranged update sounds like a disaster for compaction and read performance.

Imagine compacting or reading some SSTables in which a large number of 
overlapping but non-identical ranges were updated with different values. 
It gives me a headache by just thinking about it.


Ranged delete is much simpler, because the "value" is the same tombstone 
marker, and it also is guaranteed to expire and disappear eventually, so 
the performance impact of dealing with them at read and compaction time 
doesn't suffer in the long term.



On 14/05/2024 16:59, Benjamin Lerer wrote:
It should be like range tombstones ... in much worse ;-). A tombstone 
is a simple marker (deleted). An update can be far more complex.


Le mar. 14 mai 2024 à 15:52, Jon Haddad  a écrit :

Is there a technical limitation that would prevent a range write
that functions the same way as a range tombstone, other than
probably needing a version bump of the storage format?


On Tue, May 14, 2024 at 12:03 AM Benjamin Lerer
 wrote:

Range restrictions (>, >=, =<, < and BETWEEN) do not work on
UPDATEs. They do work on DELETE because under the hood C* they
get translated into range tombstones.

Le mar. 14 mai 2024 à 02:44, David Capwell
 a écrit :

I would also include in UPDATE… but yeah, <3 BETWEEN and
welcome this work.


On May 13, 2024, at 7:40 AM, Patrick McFadin
 wrote:

This is a great feature addition to CQL! I get
asked about it from time to time but then people figure
out a workaround. It will be great to just have it
available.

And right on Simon! I think the only project I had as a
high school senior was figuring out how many parties I
could go to and still maintain a passing grade. Thanks
for your work here.

Patrick

On Mon, May 13, 2024 at 1:35 AM Benjamin Lerer
 wrote:

Hi everybody,

Just raising awareness that Simon is working on
adding support for the BETWEEN operator in WHERE
clauses (SELECT and DELETE) in CASSANDRA-19604. We
plan to add support for it in conditions in a
separate patch.

The patch is available.

As a side note, Simon chose to do his highschool
senior project contributing to Apache Cassandra. This
patch is his first contribution for his senior
project (his second feature contribution to Apache
Cassandra).




Re: [DISCUSS] inotify for detection of manually removed snapshots

2024-08-09 Thread Bowen Song via dev

Has anyone considered simply updating the documentation saying this?

"Removing the snapshot files directly from the filesystem may break 
things. Always use the `nodetool` command or JMX to remove snapshots."


On 09/08/2024 09:18, Štefan Miklošovič wrote:
If we consider caching it all to be too much, we might probably make 
caching an option an admin would need to opt-in into? There might be a 
flag in cassandra.yaml, once enabled, it would be in memory, otherwise 
it would just load it as it was so people can decide if caching is 
enough for them or they want to have it as it was before (would be by 
default set to as it was). This puts additional complexity into 
SnapshotManager but it should be in general doable.


Let me know what you think, I would really like to have this resolved, 
18111 brings a lot of code cleanup and simplifies stuff a lot.


On Wed, Aug 7, 2024 at 11:30 PM Josh McKenzie  
wrote:



If you have a lot of snapshots and have for example a metric
monitoring them and their sizes, if you don’t cache it, creating
the metric can cause performance degradation. We added the cache
because we saw this happen to databases more than once.

I mean, I believe you, I'm just surprised querying out metadata
for files and basic computation is leading to hundreds of ms pause
times even on systems with a lot of files. Aren't most / all of
these values cached at the filesystem layer so we're basically
just tomato / tomahto caching systems, either one we maintain or
one the OS maintains?

Or is there really just a count of files well outside what I'm
thinking here?

Anyway, not trying to cause a ruckus and make needless noise,
trying to learn. ;)


On Wed, Aug 7, 2024, at 3:20 PM, Štefan Miklošovič wrote:



On Wed, Aug 7, 2024 at 6:39 PM Yifan Cai  wrote:

With WatcherService, when events are missed (which is to be
expected), you will still need to list the files. It seems to
me that WatcherService doesn't offer significant benefits in
this case.


Yeah I think we leave it out eventually.


Regarding listing directory with a refresh flag, my concern
is the potential for abuse. End-users might/could always
refresh before listing, which could undermine the purpose of
caching. Perhaps Jeremiah can provide more insight on this.


Well, by default, it would not be refreshed every single time.
You would need to opt-in into that. If there is a shop which has
users with a direct access to the disk of Cassandra nodes and
they are removing data manually, I do not know what to say, what
is nodetool clearsnapshot and jmx methods good for then? I do not
think we can prevent people from shooting into their feet if they
are absolutely willing to do that.

If they want to refresh that every time, that would be equal to
the current behavior. It would be at most as "bad" as it is now.

IMO, caching is best handled internally. I have a few
UX-related questions:
- Is it valid or acceptable to return stale data? If so,
end-users have to do some form of validation before consuming
each snapshot to account for potential deletions.


answer below

- Even if listsnapshot returns the most recent data, is it
possible that some of the directories get deleted when
end-users are accessing them? I think it is true. It, then,
enforces end-users to do some validation first, similar to
handling stale data.


I think that what you were trying to say is that when at time T0
somebody lists snapshots and at T1 somebody removes a snapshot
manually then the list of snapshots is not actual anymore? Sure.
That is a thing. This is how it currently works.

Now, we want to cache them, so if you clear a snapshot which is
not physically there because somebody removed it manually, that
should be a no-op, it will be just removed from the internal
tracker. So, if it is at disk and in cache and you clear it, then
all is fine. It is fine too if it is not on disk anymore and you
clear it, then it is just removed internally. It would fail only
in case you want to remove a snapshot which is not cached,
regardless whether it is on disk or not. The deletion of
non-existing snapshot ends up with a failure, nothing should be
changed in that regard, this is the current behavior too.

I want to say that I did not write it completely correctly at the
very beginning of this thread. Currently, we are caching only
_expiring_ snapshots, because we need to know what is their time
of removal so we act on it later. _normal_ snapshots are _not_
cached _yet_. I spent so much time with 18111 that I live in a
reality where it is already in, I forgot this is not actually in
place yet, we are very close to that.

OK thank you all for your 

Re: [DISCUSS] inotify for detection of manually removed snapshots

2024-08-09 Thread Bowen Song via dev
If we have the documentation in place, we can then consider the cache to 
be the master copy of metadata, and rely on it to be always accurate and 
up to date. If someone deletes the snapshot files from filesystem, they 
can't complain about Cassandra stopped working correctly - which is the 
same if they had manually deleted some SSTable files (they shouldn't).


On 09/08/2024 11:16, Štefan Miklošovič wrote:
We could indeed do that. Does your suggestion mean that there should 
not be a problem with caching it all once explicitly stated like that?


On Fri, Aug 9, 2024 at 12:01 PM Bowen Song via dev 
 wrote:


Has anyone considered simply updating the documentation saying this?

"Removing the snapshot files directly from the filesystem may
break things. Always use the `nodetool` command or JMX to remove
snapshots."

On 09/08/2024 09:18, Štefan Miklošovič wrote:

If we consider caching it all to be too much, we might probably
make caching an option an admin would need to opt-in into? There
might be a flag in cassandra.yaml, once enabled, it would be in
memory, otherwise it would just load it as it was so people can
decide if caching is enough for them or they want to have it as
it was before (would be by default set to as it was). This puts
additional complexity into SnapshotManager but it should be in
general doable.

Let me know what you think, I would really like to have this
resolved, 18111 brings a lot of code cleanup and simplifies stuff
a lot.

On Wed, Aug 7, 2024 at 11:30 PM Josh McKenzie
 wrote:


If you have a lot of snapshots and have for example a metric
monitoring them and their sizes, if you don’t cache it,
creating the metric can cause performance degradation. We
added the cache because we saw this happen to databases more
than once.

I mean, I believe you, I'm just surprised querying out
metadata for files and basic computation is leading to
hundreds of ms pause times even on systems with a lot of
files. Aren't most / all of these values cached at the
filesystem layer so we're basically just tomato / tomahto
caching systems, either one we maintain or one the OS maintains?

Or is there really just a count of files well outside what
I'm thinking here?

Anyway, not trying to cause a ruckus and make needless noise,
trying to learn. ;)


On Wed, Aug 7, 2024, at 3:20 PM, Štefan Miklošovič wrote:



On Wed, Aug 7, 2024 at 6:39 PM Yifan Cai
 wrote:

With WatcherService, when events are missed (which is to
be expected), you will still need to list the files. It
seems to me that WatcherService doesn't offer
significant benefits in this case.


Yeah I think we leave it out eventually.


Regarding listing directory with a refresh flag, my
concern is the potential for abuse. End-users
might/could always refresh before listing, which could
undermine the purpose of caching. Perhaps Jeremiah can
provide more insight on this.


Well, by default, it would not be refreshed every single
time. You would need to opt-in into that. If there is a shop
which has users with a direct access to the disk of
Cassandra nodes and they are removing data manually, I do
not know what to say, what is nodetool clearsnapshot and jmx
methods good for then? I do not think we can prevent people
from shooting into their feet if they are absolutely willing
to do that.

If they want to refresh that every time, that would be equal
to the current behavior. It would be at most as "bad" as it
is now.

IMO, caching is best handled internally. I have a few
UX-related questions:
- Is it valid or acceptable to return stale data? If so,
end-users have to do some form of validation before
consuming each snapshot to account for potential deletions.


answer below

- Even if listsnapshot returns the most recent data, is
it possible that some of the directories get deleted
when end-users are accessing them? I think it is true.
It, then, enforces end-users to do some validation
first, similar to handling stale data.


I think that what you were trying to say is that when at
time T0 somebody lists snapshots and at T1 somebody removes
a snapshot manually then the list of snapshots is not actual
anymore? Sure. That is a thing. This is how it currently works.

Now, we want to cache them, so if you clear a snapshot which
is not physically there because somebody removed it
manually, that should be a n

Re: 【DISCUSS】The configuration of Commitlog archiving

2024-08-30 Thread Bowen Song via dev
I'm not sure what is the concern here. Is it a malicious user exploiting 
this? Or human error with unintended consequences?


For malicious user, in order to exploit this, an attacker needs to be 
able to write to the config file. The config file on Linux by default is 
owned by the root user and has the -rw-r--r-- permission, that means the 
attacker must either gain root access to the system or has the ability 
to write arbitrary file on the filesystem. With either of these 
permission, they can already do almost anything they want (e.g. modify a 
SUID executable file). They wouldn't even need to exploit this to run a 
script or dangerous command. So this sounds like a non-issue to me, at 
least on Linux-based OSes.


For human error, if the operator puts "rm -rf" in it, the software 
should treat it as the operator actually wants to do that. I personally 
don't like software attempting to outsmart human, which often ends up 
interfering with legitimate use cases. The best thing a software can do 
is log it, so there's some traceability if and when things go wrong.


So, IMO, there's nothing wrong with the implementation in Cassandra.


On 30/08/2024 17:13, guo Maxwell wrote:
Commitlog has the ability of archive  log file, see 
CommitLogArchiver.java 
,  
we can achieve the purpose of archive and restore commitlog by 
configuring archive_command and restore_command in 
commitlog_archiving.properties 
.The 
archive_command and restore_command can be some linux/unix shell 
command.  However, I found that the shell command can actually be 
filled with any script, even if "*rm -rf"* .I have tested this 
situation and it finally succeeded with my test file being deleted.


  Personally, I think it is a dangerous behavior, because if there are 
no system-level restrictions and users are allowed to do anything in 
these shell commands. So here I want to discuss with you whether it is 
necessary to impose any restrictions on use, or do we need a new way 
of archiving/restoring commitlog?
Of course, before that, I would also like to ask, how many people are 
using archive and restore of commitlog? It seems that the commitlog 
archive code has not been updated for a long time.


I have two ideas.
One is to make some restrictions on the command context based on the 
existing usage methods, such as strictly only allowing the current 
cp/mv/ln %path to %name.Other redundant strings in the command are not 
allowed.
Another one , As I roughly investigated the archive of mysql and pg. 
They do not give users too much space (I am talking about letting 
users define their own archiving command ), and archive directly to a 
designated location. For us, I feel that we can refer to c * 
Incremental backup of sstable,  add a hardlink to the commitlog to the 
specified location, but this place may modify the original 
configuration method, such as setting the archive location and 
restoring location of the node through nodetool and deprecate the 
commitlog_archiving.properties 
 
configuration.


I am just putting forward some views  here, and looking forward to 
your feedback. 😀


Re: [DISCUSS] Improve Commitlog write path

2022-07-20 Thread Bowen Song via dev
From my past experience, the bottleneck for insert heavy workload is 
likely to be compaction, not commit log. You initially may see commit 
log as the bottleneck when the table size is relatively small, but as 
the table size increases, compaction will likely take its place and 
become the new bottleneck.


On 20/07/2022 11:11, Pawar, Amit wrote:


[Public]

Hi all,

(My previous mail is not appearing in mailing list and resending again 
after 2 days)


Myself Amit and working at AMD Bangalore, India. I am new to Cassandra 
and need to do Cassandra testing on large core systems. Usually should 
test on multi-nodes Cassandra but started with Single node testing to 
understand how Cassandra scales with increasing core counts.


Test details:

Operation: Insert > 90% (insert heavy)

Operation: Scan < 10%

Cassandra: 3.11.10 and trunk

Benchmark: TPCx-IOT (similar to YCSB)

Results shows scaling is poor beyond 16 cores and it is almost linear. 
Following settings are the common settings helped to get the better 
scores.


 1. Memtable heap allocation: offheap_objects
 2. memtable_flush_writers > 4
 3. Java heap: 8-32GB with survivor ratio tuning
 4. Separate storage space for Commitlog and Data.

Many online blogs suggest to add new Cassandra node when unable to 
take high writes. But with large systems, high writes should be easily 
taken due to many cores. Need was to improve the scaling with more 
cores so this suggestion didn’t help. After many rounds of testing it 
was observed that current implementation uses single thread for 
Commitlog syncing activity. Commitlog files are mapped using mmap 
system call and changes are written with msync. Periodic syncing with 
JVisualvm tool shows


 1. thread is not 100% busy with Ramdisk usage for Commitlog storage
and scaling improved on large systems. Ramdisk scores > 2 X NVME
score.
 2. thread becomes 100% busy with NVME usage for Commiglog and score
does not improve much beyond 16 cores.

Linux kernel uses 4K pages for mapped memory with mmap system call. 
So, to understand this further, disk I/O testing was done using FIO 
tool and results shows


 1. NVME 4K random R/W throughput is very less with single thread and
it improves with multi-threaded.
 2. Ramdisk 4K random R/W throughput is good with single thread only
and also better with multi-threaded

Based on the FIO test results following two ideas were tested for 
Commitlog files with Cassandra-3.1.10 sources.


 1. Enable Direct IO feature for Commitlog files (similar to
[CASSANDRA-14466] Enable Direct I/O - ASF JIRA (apache.org)
 )
 2. Enable Multi-threaded syncing for Commitlog files.

First one need to retest. Interestingly second one helped to improve 
the score with “NVME” disk. NVME disk configuration score is almost 
within 80-90% of ramdisk and 2 times of single threaded 
implementation. Multithreading enabled by adding new thread pool in 
“AbstractCommitLogSegmentManager” class and changed syncing thread as 
manager thread for this new thread pool to take care synchronization. 
Only tested with Cassandra-3.11.10 and needs complete testing but this 
change is working in my test environment. Tried these few experiments 
so that I could discuss here and seek your valuable suggestions to 
identify the right fix for insert heavy workloads.


 1. Is it good idea to convert single threaded syncing to
multi-threading implementation to improve the disk IO?
 2. Direct I/O throughput is high with single thread and best fit for
Commitlog case due to file size. This will improve writes on small
to large systems. Good to bring this support for Commitlog files?

Please suggest.

Thanks,

Amit Pawar


Re: [DISCUSS] Improve Commitlog write path

2022-07-22 Thread Bowen Song via dev

Hi Amit,


The compaction bottleneck is not an instantly visible limitation. It in 
effect limits the total size of writes over a fairly long period of 
time, because compaction is asynchronous and can be queued. That means 
if compaction can't keep up with the writes, they will be queued, and 
Cassandra remains fully functional until hitting the "too many open 
files" error or the filesystem runs out of free inodes. This can happen 
over many days or even weeks.


For the purpose of benchmarking, you may prefer to measure the max 
concurrent compaction throughput, instead of actually waiting for that 
breaking moment. The max write throughput is a fraction of the max 
concurrent compaction throughput, usually by a factor of 5 or more for a 
non-trivial sized table, depending on the table size in bytes. Search 
for "STCS write amplification" to understand why that's the case. That 
means if you've measured the max concurrent compaction throughput is 
1GB/s, your average max insertion speed over a period of time is 
probably less than 200MB/s.


If you really decide to test the compaction bottleneck in action, it's 
better to measure the table size in bytes on disk, rather than the 
number of records. That's because not only the record count, but also 
the size of partitions and compression ratio, all have meaningful effect 
on the compaction workload. It's also worth mentioning that if using the 
STCS strategy, which is more suitable for write heavy workload, you may 
want to keep an eye on the SSTable data file size distribution. 
Initially the compaction may not involve any large SSTable data file, so 
it won't be a bottleneck at all. As more bigger SSTable data files are 
created over time, they will get involved in compactions more and more 
frequently. The bottleneck will only shows up (i.e. become problematic) 
when there's sufficient number of large SSTable data files involved in 
multiple concurrent compactions, occupying all available compactors and 
blocks (queuing) a larger number of compactions involving smaller 
SSTable data files.



Regards,

Bowen


On 22/07/2022 11:19, Pawar, Amit wrote:


[Public]

Thank you Bowen for your reply. Took some time to respond due to 
testing issue.


I tested again multi-threaded feature with number of records from 260 
million to 2 billion and still improvement is seen around 80% of 
Ramdisk score. It is still possible that compaction can become new 
bottleneck and could be new opportunity to fix it. I am newbie here 
and possible that I failed to understand your suggestion completely. 
 At-least with this testing multi-threading benefit is reflecting in 
score.


Do you think multi-threading is good to have now ? else please suggest 
if I need to test further.


Thanks,

Amit

*From:* Bowen Song via dev 
*Sent:* Wednesday, July 20, 2022 4:13 PM
*To:* dev@cassandra.apache.org
*Subject:* Re: [DISCUSS] Improve Commitlog write path

[CAUTION: External Email]

From my past experience, the bottleneck for insert heavy workload is 
likely to be compaction, not commit log. You initially may see commit 
log as the bottleneck when the table size is relatively small, but as 
the table size increases, compaction will likely take its place and 
become the new bottleneck.


On 20/07/2022 11:11, Pawar, Amit wrote:

[Public]

Hi all,

(My previous mail is not appearing in mailing list and resending
again after 2 days)

Myself Amit and working at AMD Bangalore, India. I am new to
Cassandra and need to do Cassandra testing on large core systems.
Usually should test on multi-nodes Cassandra but started with
Single node testing to understand how Cassandra scales with
increasing core counts.

Test details:

Operation: Insert > 90% (insert heavy)

Operation: Scan < 10%

Cassandra: 3.11.10 and trunk

Benchmark: TPCx-IOT (similar to YCSB)

Results shows scaling is poor beyond 16 cores and it is almost
linear. Following settings are the common settings helped to get
the better scores.

 1. Memtable heap allocation: offheap_objects
 2. memtable_flush_writers > 4
 3. Java heap: 8-32GB with survivor ratio tuning
 4. Separate storage space for Commitlog and Data.

Many online blogs suggest to add new Cassandra node when unable to
take high writes. But with large systems, high writes should be
easily taken due to many cores. Need was to improve the scaling
with more cores so this suggestion didn’t help. After many rounds
of testing it was observed that current implementation uses single
thread for Commitlog syncing activity. Commitlog files are mapped
using mmap system call and changes are written with msync.
Periodic syncing with JVisualvm tool shows

 1. thread is not 100% busy with Ramdisk usage for Commitlog
storage and scaling improved on large systems. Ramdisk scores
> 2 X NV

Re: [DISCUSS] Improve Commitlog write path

2022-07-26 Thread Bowen Song via dev

Hi Amit,

That's some brilliant tests you have done there. It shows that the 
compaction throughput not only can be a bottleneck on the speed of 
insert operations, but it can also stress the JVM garbage collector. As 
a result of GC pressure, it can cause other things, such as insert, to fail.


Your last statement is correct. The commit log change can be beneficial 
for atypical workloads where large volume of data is getting inserted 
and then expired soon, for example when using the 
TimeWindowCompactionStrategy with short TTL. But I must point out that 
this kind of atypical usage is often an anti-pattern in Cassandra, as 
Cassandra is a database, not a queue or cache system.


This, however, is not saying the commit log change should not be 
introduced. As others have pointed out, it's down to a balancing act 
between the cost and benefit, and it will depend on the code complexity 
and the effect it has on typical workload, such as CPU and JVM heap 
usage. After all, we should prioritise the performance and reliability 
of typical usage before optimising for atypical use cases.


Best,
Bowen

On 26/07/2022 12:41, Pawar, Amit wrote:


[Public]


Hi Bowen,

Thanks for the reply and it helped to identify the failure point. 
Tested compaction throughput with different values and threads active 
in compaction reports “java.lang.OutOfMemoryError: Map failed” error 
with 1024 MB/s earlier compared to other values. This shows with lower 
throughput such issues are going to come up not immediately but in 
days or weeks. Test results are given below.


|++---+---+-+|

|| Records    | Compaction Throughput | 5 large files In GB | Disk 
usage (GB) ||


|++---+---+-+|

|| 20 | 8 | Not collected | 500 ||

|++---+---+-+|

|| 20 | 16    | Not collected | 500 ||

|++---+---+-+|

|| 9  | 64    | 3.5,3.5,3.5,3.5,3.5 | 
273     ||


|++---+---+-+|

|| 9  | 128   | 3.5, 3.9,4.9,8.0, 15 | 
287 ||


|++---+---+-+|

|| 9  | 256   | 11,11,12,16,20 | 
359 ||


|++---+---+-+|

|| 9  | 512   | 14,19,23,27,28 | 
469 ||


|++---+---+-+|

|| 9  | 1024  | 14,18,23,27,28 | 
458 ||


|++---+---+-+|

|| 9  | 0 | 6.9,6.9,7.0,28,28 | 
223 ||


|++---+---+-+|

|| |       | | ||

|++---+---+-+|

Issues observed with increasing compaction throughput.

 1. Out of memory errors
 2. Scores reduces as throughput increased
 3. Files size grows as throughput increased
 4. Insert failures are noticed

After this testing, I feel that this change is beneficial for 
workloads where data is not kept/left on nodes for too long. With 
lower throughput large system can ingest more data. Does it make sense ?


Thanks,

Amit

*From:* Bowen Song via dev 
*Sent:* Friday, July 22, 2022 4:37 PM
*To:* dev@cassandra.apache.org
*Subject:* Re: [DISCUSS] Improve Commitlog write path

[CAUTION: External Email]

Hi Amit,

The compaction bottleneck is not an instantly visible limitation. It 
in effect limits the total size of writes over a fairly long period of 
time, because compaction is asynchronous and can be queued. That means 
if compaction can't keep up with the writes, they will be queued, and 
Cassandra remains fully functional until hitting the "too many open 
files" error or the filesystem runs out of free inodes. This can 
happen over many days or even weeks.


For the purpose of benchmarking, you may prefer to measure the max 
concurrent compaction throughput, instead of actually waiting for that 
breaking moment. The max write throughput is a fraction of the max 
concurrent compaction throughput, usually by a factor of 5 or more for 
a non-trivial sized table, depending on the table size in bytes. 
Search for "STCS write amplification" to understand why that's the 
case. That means if you've measured the max concurrent compaction 
throughput is 1GB/s, your average max insertion speed over a period of 
time is probably less than 200MB/s.


If you reall

Re: [DISCUSS] Deprecate and remove resumable bootstrap and decommission

2022-08-03 Thread Bowen Song via dev
I have benefited from the resumable bootstrap before, and I'm in favour 
of keeping the feature around.


I've had streaming failures due to long STW GC pauses on some 
bootstrapping nodes, and I had to resume the bootstrap once or twice in 
order to get these nodes finish joinning the cluster. They had not 
experienced more long STW GC pauses since they joined the cluster. I 
would imagine I will spend a lots of time tuning the GC parameters in 
order get these nodes to join if the resumable bootstrapping feature is 
removed. Also, I'm not concerned about racing conditions involving 
repairs, because we don't run repairs while we are adding new nodes (to 
minimize the additional load on the cluster).



On 03/08/2022 19:46, Josh McKenzie wrote:

Context: https://issues.apache.org/jira/browse/CASSANDRA-17679

From the .yaml comment on the param I was working on adding:
In certain environments, operators may want to disable resumable bootstrap in 
order to avoid potential correctness violations or data loss scenarios. 
Largelythis  centers around nodes going down during bootstrap, tombstones being 
written, and potential races with repair. Bydefault  we leavethis  on as it's 
been enabledfor  quite some time, however the option to disable it is more 
palatable now that we have zero copy streaming as that greatly accelerates

Given zero copy streaming in the system and the general unexplored 
correctness concerns of 
https://issues.apache.org/jira/browse/CASSANDRA-8838, specifically 
pointed out by Jeff here: 
https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234 
, I've 
been chatting w/Paulo about this and we've both concluded we think the 
functionality should be made configurable, default off (?), deprecated 
in 4.2 and then completely removed next.


- First: anyone have any concerns with the general arc of "remove 
resumable bootstrap and decommission"?

- Second: Should we leave them enabled by default in 4.2 or disabled?
- Third: Should we consider revisiting older branches with this 
functionality and making it toggle-able?


~Josh

Re: [DISCUSS] Deprecate and remove resumable bootstrap and decommission

2022-08-03 Thread Bowen Song via dev

That would have to be assessed on a case by case basis.

* When the code doesn't delete data, which means there's a zero 
probability of resurrecting deleted data, I will still use resumable 
bootstrap.


* When resurrected data doesn't pose a problem to the system, it often 
can still be an acceptable behaviour to save hours or days of 
bootstrapping time. I may use resumable bootstrap.


* In other cases, where data correctness is important and there's a 
chance for resurrecting deleted data, I would certainly not use it if I 
had known it in advance (which I don't).



On 03/08/2022 23:11, Jeff Jirsa wrote:
The hypothetical concern described is around potential data 
resurrection - would you still use resumable bootstrap if you knew 
that data deleted during those STW pauses was improperly resurrected?


On Wed, Aug 3, 2022 at 2:40 PM Bowen Song via dev 
 wrote:


I have benefited from the resumable bootstrap before, and I'm in
favour of keeping the feature around.

I've had streaming failures due to long STW GC pauses on some
bootstrapping nodes, and I had to resume the bootstrap once or
twice in order to get these nodes finish joinning the cluster.
They had not experienced more long STW GC pauses since they joined
the cluster. I would imagine I will spend a lots of time tuning
the GC parameters in order get these nodes to join if the
resumable bootstrapping feature is removed. Also, I'm not
concerned about racing conditions involving repairs, because we
don't run repairs while we are adding new nodes (to minimize the
additional load on the cluster).


On 03/08/2022 19:46, Josh McKenzie wrote:

Context: https://issues.apache.org/jira/browse/CASSANDRA-17679

From the .yaml comment on the param I was working on adding:
In certain environments, operators may want to disable resumable bootstrap 
in order to avoid potential correctness violations or data loss scenarios. 
Largelythis  centers around nodes going down during bootstrap, tombstones being 
written, and potential races with repair. Bydefault  we leavethis  on as it's 
been enabledfor  quite some time, however the option to disable it is more 
palatable now that we have zero copy streaming as that greatly accelerates

Given zero copy streaming in the system and the general
unexplored correctness concerns of
https://issues.apache.org/jira/browse/CASSANDRA-8838,
specifically pointed out by Jeff here:

https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234

<https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234>,
 I've
been chatting w/Paulo about this and we've both concluded we
think the functionality should be made configurable, default off
(?), deprecated in 4.2 and then completely removed next.

- First: anyone have any concerns with the general arc of "remove
resumable bootstrap and decommission"?
- Second: Should we leave them enabled by default in 4.2 or disabled?
- Third: Should we consider revisiting older branches with this
functionality and making it toggle-able?

~Josh


Re: [DISCUSS] Deprecate and remove resumable bootstrap and decommission

2022-08-03 Thread Bowen Song via dev
That was Cassandra 3.11, before the introduction of zero copy. But I 
must say I'm not certain whether the new zero copy streaming can prevent 
the long GC pauses, as I haven't tried it.


On 03/08/2022 23:37, Josh McKenzie wrote:
I had to resume the bootstrap once or twice in order to get these 
nodes finish joinning the cluster.
Was this before or after the addition of zero copy streaming? The 
premise is that the pain point resumable bootstrap targets is 
mitigated by the much faster bootstrapping times without the 
correctness risks.


On Wed, Aug 3, 2022, at 6:21 PM, Bowen Song via dev wrote:


That would have to be assessed on a case by case basis.

* When the code doesn't delete data, which means there's a zero 
probability of resurrecting deleted data, I will still use resumable 
bootstrap.


* When resurrected data doesn't pose a problem to the system, it 
often can still be an acceptable behaviour to save hours or days of 
bootstrapping time. I may use resumable bootstrap.


* In other cases, where data correctness is important and there's a 
chance for resurrecting deleted data, I would certainly not use it if 
I had known it in advance (which I don't).



On 03/08/2022 23:11, Jeff Jirsa wrote:
The hypothetical concern described is around potential data 
resurrection - would you still use resumable bootstrap if you knew 
that data deleted during those STW pauses was improperly resurrected?


On Wed, Aug 3, 2022 at 2:40 PM Bowen Song via dev 
mailto:dev@cassandra.apache.org>> wrote:


I have benefited from the resumable bootstrap before, and I'm in
favour of keeping the feature around.

I've had streaming failures due to long STW GC pauses on some
bootstrapping nodes, and I had to resume the bootstrap once or
twice in order to get these nodes finish joinning the cluster.
They had not experienced more long STW GC pauses since they
joined the cluster. I would imagine I will spend a lots of time
tuning the GC parameters in order get these nodes to join if the
resumable bootstrapping feature is removed. Also, I'm not
concerned about racing conditions involving repairs, because we
don't run repairs while we are adding new nodes (to minimize the
additional load on the cluster).


On 03/08/2022 19:46, Josh McKenzie wrote:

Context: https://issues.apache.org/jira/browse/CASSANDRA-17679
<https://issues.apache.org/jira/browse/CASSANDRA-17679>

From the .yaml comment on the param I was working on adding:
In certain environments, operators may want to disable resumable bootstrap 
in order to avoid potential correctness violations or data loss scenarios. 
Largelythis  centers around nodes going down during bootstrap, tombstones being 
written, and potential races with repair. Bydefault  we leavethis  on as it's 
been enabledfor  quite some time, however the option to disable it is more 
palatable now that we have zero copy streaming as that greatly accelerates


Given zero copy streaming in the system and the general
unexplored correctness concerns of
https://issues.apache.org/jira/browse/CASSANDRA-8838
<https://issues.apache.org/jira/browse/CASSANDRA-8838>,
specifically pointed out by Jeff here:

https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234

<https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234>,
 I've
been chatting w/Paulo about this and we've both concluded we
think the functionality should be made configurable, default
off (?), deprecated in 4.2 and then completely removed next.

- First: anyone have any concerns with the general arc of
"remove resumable bootstrap and decommission"?
- Second: Should we leave them enabled by default in 4.2 or
disabled?
- Third: Should we consider revisiting older branches with this
functionality and making it toggle-able?

~Josh




Re: [DISCUSS] Deprecate and remove resumable bootstrap and decommission

2022-08-03 Thread Bowen Song via dev
I should also add that because we use vnodes and STCS, in the absent of 
CASSANDRA-10540 <https://issues.apache.org/jira/browse/CASSANDRA-10540>, 
I don't think we will benefit from the zero copy streaming at all, as 
almost all SSTables files from the streaming source will contain a very 
wide token range outside the receiving node's desired token range.


On 04/08/2022 00:41, Bowen Song wrote:


That was Cassandra 3.11, before the introduction of zero copy. But I 
must say I'm not certain whether the new zero copy streaming can 
prevent the long GC pauses, as I haven't tried it.


On 03/08/2022 23:37, Josh McKenzie wrote:
I had to resume the bootstrap once or twice in order to get these 
nodes finish joinning the cluster.
Was this before or after the addition of zero copy streaming? The 
premise is that the pain point resumable bootstrap targets is 
mitigated by the much faster bootstrapping times without the 
correctness risks.


On Wed, Aug 3, 2022, at 6:21 PM, Bowen Song via dev wrote:


That would have to be assessed on a case by case basis.

* When the code doesn't delete data, which means there's a zero 
probability of resurrecting deleted data, I will still use resumable 
bootstrap.


* When resurrected data doesn't pose a problem to the system, it 
often can still be an acceptable behaviour to save hours or days of 
bootstrapping time. I may use resumable bootstrap.


* In other cases, where data correctness is important and there's a 
chance for resurrecting deleted data, I would certainly not use it 
if I had known it in advance (which I don't).



On 03/08/2022 23:11, Jeff Jirsa wrote:
The hypothetical concern described is around potential data 
resurrection - would you still use resumable bootstrap if you knew 
that data deleted during those STW pauses was improperly resurrected?


On Wed, Aug 3, 2022 at 2:40 PM Bowen Song via dev 
mailto:dev@cassandra.apache.org>> wrote:


I have benefited from the resumable bootstrap before, and I'm
in favour of keeping the feature around.

I've had streaming failures due to long STW GC pauses on some
bootstrapping nodes, and I had to resume the bootstrap once or
twice in order to get these nodes finish joinning the cluster.
They had not experienced more long STW GC pauses since they
joined the cluster. I would imagine I will spend a lots of time
tuning the GC parameters in order get these nodes to join if
the resumable bootstrapping feature is removed. Also, I'm not
concerned about racing conditions involving repairs, because we
don't run repairs while we are adding new nodes (to minimize
the additional load on the cluster).


On 03/08/2022 19:46, Josh McKenzie wrote:

Context: https://issues.apache.org/jira/browse/CASSANDRA-17679
<https://issues.apache.org/jira/browse/CASSANDRA-17679>

From the .yaml comment on the param I was working on adding:
In certain environments, operators may want to disable resumable bootstrap 
in order to avoid potential correctness violations or data loss scenarios. 
Largelythis  centers around nodes going down during bootstrap, tombstones being 
written, and potential races with repair. Bydefault  we leavethis  on as it's 
been enabledfor  quite some time, however the option to disable it is more 
palatable now that we have zero copy streaming as that greatly accelerates


Given zero copy streaming in the system and the general
unexplored correctness concerns of
https://issues.apache.org/jira/browse/CASSANDRA-8838
<https://issues.apache.org/jira/browse/CASSANDRA-8838>,
specifically pointed out by Jeff here:

https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234

<https://issues.apache.org/jira/browse/CASSANDRA-8838?focusedCommentId=16900234&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16900234>,
 I've
been chatting w/Paulo about this and we've both concluded we
think the functionality should be made configurable, default
off (?), deprecated in 4.2 and then completely removed next.

- First: anyone have any concerns with the general arc of
"remove resumable bootstrap and decommission"?
- Second: Should we leave them enabled by default in 4.2 or
disabled?
- Third: Should we consider revisiting older branches with
this functionality and making it toggle-able?

~Josh




Re: Unsubscribe

2022-08-09 Thread Bowen Song via dev
To unsubscribe from this mailing list, you'll need to send an email to 
dev-unsubscr...@cassandra.apache.org


On 09/08/2022 12:52, Schmidtberger, Brian M. (STL) wrote:


unsubscribe

+

BRIAN SCHMIDTBERGER

Software Engineering Senior Advisor, Core Engineering, Express Scripts

M: 785.766.7450

EVERNORTH.COM 

/Confidential, unpublished property of Evernorth. Do not duplicate or 
distribute. Use and distribution limited solely to authorized 
personnel. © Copyright 2022 Evernorth. _Legal Disclaimer 
_/


Re: [PROPOSAL] Moving deb/rpm repositories from downloads.apache.org to apache.jfrog.io

2022-08-11 Thread Bowen Song via dev
> /These repositories and their binaries are "convenience binaries" and 
not the official Cassandra source binaries/


Then where are the official binaries?


On 11/08/2022 21:40, Mick Semb Wever wrote:


The proposal is to move our official debian and redhat repositories 
from downloads.apache.org  to Apache's 
JFrog Artifactory server at apache.jfrog.io  , 
fronting it with the url aliases debian.cassandra.apache.org 
 and redhat.cassandra.apache.org 



That is to replace the following URLs from
https://downloads.apache.org/cassandra/debian/
https://downloads.apache.org/cassandra/redhat/

to
https://debian.cassandra.apache.org

https://redhat.cassandra.apache.org


(which in turn redirect to our jfrog repositories at)
https://apache.jfrog.io/artifactory/cassandra-deb
https://apache.jfrog.io/artifactory/cassandra-rpm


The rationale to do this is to avoid the strict checksum and signature 
requirements on downloads.a.o (which is the same as dist.a.o), as the 
debian and redhat repositories have their own system for integrity and 
signing (which we already do).


These repositories and their binaries are "convenience binaries" and 
not the official Cassandra source binaries, so they do not need to be 
on downloads.a.o and can be served from apache.jfrog.io 
. This is similar to maven binaries (and 
docker images).


This will BREAK everyone's existing 
`/etc/apt/sources.list.d/cassandra.sources.list` and 
`/etc/yum.repos.d/cassandra.repo` files. Folk will need to update 
these files to point to the new repo URLs.


The plan is to do the following to ensure people are informed about 
this breaking change:

 - announcement to users@
 - README.md in the original URL locations explaining the breakage and 
how to fix. (The README.md must be voted on, signed and checksummed),

 - A warning banner on our website downloads page,
 - Every release email for the next 12 months will contain the warning.


background: https://issues.apache.org/jira/browse/CASSANDRA-17748

Anyone with any questions/objections?


Re: [PROPOSAL] Moving deb/rpm repositories from downloads.apache.org to apache.jfrog.io

2022-08-11 Thread Bowen Song via dev
I'm a bit unclear what's the scope of this change. Is it limited to the 
"*-bin.tar.gz" files only?


I would assume the RPM/DEB packages are considered as parts of the 
"official releases", and aren't affected by this change. Am I right?



On 11/08/2022 21:59, Mick Semb Wever wrote:


> /These repositories and their binaries are "convenience
binaries" and not the official Cassandra source binaries/

Then where are the official binaries?



Wrong wording there., thanks for catching me.
The official *releases* are the source artefacts, e.g. the 
*-src.tar.gz in https://downloads.apache.org/cassandra/4.0.5/


The binaries (e.g. *-bin.tar.gz) are not considered official, but 
convenience.


https://infra.apache.org/release-distribution.html#release-content
https://www.apache.org/legal/release-policy.html#artifacts




Re: [PROPOSAL] Moving deb/rpm repositories from downloads.apache.org to apache.jfrog.io

2022-08-11 Thread Bowen Song via dev
In that case, the move from signed RPM/DEB to unsigned can be quiet 
problematic to some enterprise users.


On 11/08/2022 22:16, Jeremiah D Jordan wrote:
For ASF project the binary release are always considered as 
“convenience binaries”, the official release is always just the source 
artifacts.  See the ASF release policy for more information.


https://www.apache.org/legal/release-policy.html#compiled-packages


On Aug 11, 2022, at 4:12 PM, Bowen Song via dev 
 wrote:


I'm a bit unclear what's the scope of this change. Is it limited to 
the "*-bin.tar.gz" files only?


I would assume the RPM/DEB packages are considered as parts of the 
"official releases", and aren't affected by this change. Am I right?



On 11/08/2022 21:59, Mick Semb Wever wrote:


> /These repositories and their binaries are "convenience
binaries" and not the official Cassandra source binaries/

Then where are the official binaries?



Wrong wording there., thanks for catching me.
The official *releases* are the source artefacts, e.g. the 
*-src.tar.gz in https://downloads.apache.org/cassandra/4.0.5/


The binaries (e.g. *-bin.tar.gz) are not considered official, but 
convenience.


https://infra.apache.org/release-distribution.html#release-content
https://www.apache.org/legal/release-policy.html#artifacts





Re: [PROPOSAL] Moving deb/rpm repositories from downloads.apache.org to apache.jfrog.io

2022-08-11 Thread Bowen Song via dev
I see. Now I fully understand the change. There's no objections from me, 
everything sounds fine.


We should have the new domain/URL created before the final move is made, 
and redirecting to the existing download.apache.org for the time being. 
This will ensure users can have a transition time and avoid causing a 
cliff edge moment.



On 11/08/2022 22:24, Brandon Williams wrote:

Nothing is changing in regard to signing.  Both package management
systems have their own system for that which will remain.  The package
locations are being moved because downloads.apache.org wants another
level of (superfluous) signing on top of that, which we do not
currently have.

Kind Regards,
Brandon

On Thu, Aug 11, 2022 at 4:20 PM Bowen Song via dev
 wrote:

In that case, the move from signed RPM/DEB to unsigned can be quiet problematic 
to some enterprise users.

On 11/08/2022 22:16, Jeremiah D Jordan wrote:

For ASF project the binary release are always considered as “convenience 
binaries”, the official release is always just the source artifacts.  See the 
ASF release policy for more information.

https://www.apache.org/legal/release-policy.html#compiled-packages


On Aug 11, 2022, at 4:12 PM, Bowen Song via dev  
wrote:

I'm a bit unclear what's the scope of this change. Is it limited to the 
"*-bin.tar.gz" files only?

I would assume the RPM/DEB packages are considered as parts of the "official 
releases", and aren't affected by this change. Am I right?


On 11/08/2022 21:59, Mick Semb Wever wrote:



These repositories and their binaries are "convenience binaries" and not the 
official Cassandra source binaries

Then where are the official binaries?



Wrong wording there., thanks for catching me.
The official *releases* are the source artefacts, e.g. the *-src.tar.gz in 
https://downloads.apache.org/cassandra/4.0.5/

The binaries (e.g. *-bin.tar.gz) are not considered official, but convenience.

https://infra.apache.org/release-distribution.html#release-content
https://www.apache.org/legal/release-policy.html#artifacts






Re: [PROPOSAL] Moving deb/rpm repositories from downloads.apache.org to apache.jfrog.io

2022-08-11 Thread Bowen Song via dev

I see. In that case, stick to the original plan makes more sense.

On 11/08/2022 22:46, Mick Semb Wever wrote:


We should have the new domain/URL created before the final move is
made,
and redirecting to the existing download.apache.org
 for the time being.
This will ensure users can have a transition time and avoid causing a
cliff edge moment.


Good idea, but in this situation it would only complicate things, 
because (but mostly (3))
1. The jfrog repositories already exist, and have for a while now (we 
just have not publicised them so much).
2. The new URLs are already in place, redirecting to the jfrog 
repositories.
3. ASF Infra is requesting we remove the rpm/deb files from 
downloads.a.o asap.





Unexplained stuck memtable flush

2024-11-05 Thread Bowen Song via dev

Hi all,

We have a cluster running Cassandra 4.1.1. We are seeing the memtable 
flush randomly getting stuck. This has happened twice in the last 10 
days, to two different nodes in the same cluster. This started to happen 
after we enabled CDC, and each time it got stuck, there was at least one 
repair running involving the affected node.


The signs of the stuck memtable flush is most obvious from the 
"StatusLogger" logs.


At the beginning, the MemtablePostFlush and MemtableFlushWriter got 
stuck, they have 1 and 2 active tasks each, and a small number of 
pending tasks.


INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:65 - 
Pool Name   Active   Pending  Completed Blocked  
All Time Blocked
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
ReadStage 0 0   34052333 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
CompactionExecutor 0 0    1019777 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
MutationStage 0 0   14930764 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
MemtableReclaimMemory 0 0  21877 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
PendingRangeCalculator 0 0    177 
0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
Repair#61 0 0   1344 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
GossipStage 0 0 889452 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
SecondaryIndexManagement 0 0  1 
0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
HintsDispatcher 0 0 19 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
Repair-Task 0 0 65 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
RequestResponseStage 0 0   7834 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
Native-Transport-Requests 0 0    8967921 
0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
MigrationStage 0 0  5 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
MemtableFlushWriter 2    10  21781 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
MemtablePostFlush 1    11  47856 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
PerDiskMemtableFlushWriter_0 0 0  21769 
0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
Sampler 0 0  0 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
ValidationExecutor 0 0  36651 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
ViewBuildExecutor 0 0  0 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
InternalResponseStage 0 0    240 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
AntiEntropyStage 1  1773 120067 0 0
INFO  [ScheduledTasks:1] 2024-11-03 03:50:15,224 StatusLogger.java:69 - 
CacheCleanupExecutor 0 0  0 0 0


The number of pending tasks slowly grows larger over time, and the 
number of completed tasks do not increase at all.


INFO  [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:65 - 
Pool Name   Active   Pending  Completed Blocked  
All Time Blocked
INFO  [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - 
ReadStage 0 0   39905462 0 0
INFO  [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - 
CompactionExecutor 0 0    1170100 0 0
INFO  [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - 
MutationStage 0 0   16976992 0 0
INFO  [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - 
Repair#76 0 0  0 0 0
INFO  [ScheduledTasks:1] 2024-11-03 16:33:05,941 StatusLogger.java:69 - 
Repair#74 0 0  0 0 0
INFO  [ScheduledTasks:1] 2024-11-03 16:33:05