[jira] [Comment Edited] (CASSANDRA-16047) Potential race condition in creating hard link when incremental backup is turned on

2020-08-13 Thread Wei Deng (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-16047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177122#comment-17177122
 ] 

Wei Deng edited comment on CASSANDRA-16047 at 8/13/20, 4:10 PM:


The version was mentioned in the description: 3.0.15.

The deployment is on a public cloud environment with EBS-like disks that are 
backed by SSD with decent latency, throughput and IOPS, so it is hard to think 
the culprit being in the OS and IO layer.


was (Author: weideng):
The version was mentioned in the description: 3.0.15.

> Potential race condition in creating hard link when incremental backup is 
> turned on
> ---
>
> Key: CASSANDRA-16047
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16047
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Wei Deng
>Priority: Urgent
> Attachments: incremental_backup_hardlink_exception.jpg, 
> incremental_backup_hardlink_exception1.jpg
>
>
> It seems that there is a race condition in creating hard link if incremental 
> backup is turned on.
> The following screenshot was captured in a production cluster running 
> Cassandra 3.0.15 after turning on incremental backup. When this 
> {{NoSuchFileException}} happens, due to the {{FSWriteError}} and the default 
> disk failure policy, the JVM will be shutdown, so it's a pretty critical bug.
>  !incremental_backup_hardlink_exception.jpg!
> Due to the risk of causing production database downtime (if similar issue 
> happens on multiple nodes in a short time frame), and same exception causing 
> JVM shutdown multiple times already, incremental backup had to be turned off 
> for now, but this is not an ideal situation.
> !incremental_backup_hardlink_exception1.jpg!
> The deployment is on a public cloud environment with EBS-like disks that are 
> backed by SSD with decent latency, throughput and IOPS, so it is hard to 
> think the culprit being in the OS and IO layer. Based on the second 
> screenshot above, this is a low flush traffic {{system.size_estimates}} 
> table, so compaction of the source SSTable doesn't seem to be at play here.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-16047) Potential race condition in creating hard link when incremental backup is turned on

2020-08-13 Thread Wei Deng (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-16047?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17177122#comment-17177122
 ] 

Wei Deng commented on CASSANDRA-16047:
--

The version was mentioned in the description: 3.0.15.

> Potential race condition in creating hard link when incremental backup is 
> turned on
> ---
>
> Key: CASSANDRA-16047
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16047
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Wei Deng
>Priority: Urgent
> Attachments: incremental_backup_hardlink_exception.jpg, 
> incremental_backup_hardlink_exception1.jpg
>
>
> It seems that there is a race condition in creating hard link if incremental 
> backup is turned on.
> The following screenshot was captured in a production cluster running 
> Cassandra 3.0.15 after turning on incremental backup. When this 
> {{NoSuchFileException}} happens, due to the {{FSWriteError}} and the default 
> disk failure policy, the JVM will be shutdown, so it's a pretty critical bug.
>  !incremental_backup_hardlink_exception.jpg!
> Due to the risk of causing production database downtime (if similar issue 
> happens on multiple nodes in a short time frame), and same exception causing 
> JVM shutdown multiple times already, incremental backup had to be turned off 
> for now, but this is not an ideal situation.
> !incremental_backup_hardlink_exception1.jpg!
> The deployment is on a public cloud environment with EBS-like disks that are 
> backed by SSD with decent latency, throughput and IOPS, so it is hard to 
> think the culprit being in the OS and IO layer. Based on the second 
> screenshot above, this is a low flush traffic {{system.size_estimates}} 
> table, so compaction of the source SSTable doesn't seem to be at play here.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-16047) Potential race condition in creating hard link when incremental backup is turned on

2020-08-13 Thread Wei Deng (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-16047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-16047:
-
Description: 
It seems that there is a race condition in creating hard link if incremental 
backup is turned on.

The following screenshot was captured in a production cluster running Cassandra 
3.0.15 after turning on incremental backup. When this {{NoSuchFileException}} 
happens, due to the {{FSWriteError}} and the default disk failure policy, the 
JVM will be shutdown, so it's a pretty critical bug.
 !incremental_backup_hardlink_exception.jpg!

Due to the risk of causing production database downtime (if similar issue 
happens on multiple nodes in a short time frame), and same exception causing 
JVM shutdown multiple times already, incremental backup had to be turned off 
for now, but this is not an ideal situation.

!incremental_backup_hardlink_exception1.jpg!

The deployment is on a public cloud environment with EBS-like disks that are 
backed by SSD with decent latency, throughput and IOPS, so it is hard to think 
the culprit being in the OS and IO layer. Based on the second screenshot above, 
this is a low flush traffic {{system.size_estimates}} table, so compaction of 
the source SSTable doesn't seem to be at play here.

  was:
It seems that there is a race condition in creating hard link if incremental 
backup is turned on.

The following screenshot was captured in a production cluster running Cassandra 
3.0.15 after turning on incremental backup. When this {{NoSuchFileException}} 
happens, due to the {{FSWriteError}} and the default disk failure policy, the 
JVM will be shutdown, so it's a pretty critical bug.
 !incremental_backup_hardlink_exception.jpg!

Due to the risk of causing production database downtime (if similar issue 
happens on multiple nodes in a short time frame), and similar issue causing JVM 
shutdown multiple times already, incremental backup had to be turned off for 
now, but this is not an ideal situation.

!incremental_backup_hardlink_exception1.jpg!

The deployment is on a public cloud environment with EBS-like disks that are 
backed by SSD with decent latency, throughput and IOPS, so it is hard to think 
the culprit being in the OS and IO layer. Based on the second screenshot above, 
this is a low flush traffic {{system.size_estimates}} table, so compaction of 
the source SSTable doesn't seem to be at play here.


> Potential race condition in creating hard link when incremental backup is 
> turned on
> ---
>
> Key: CASSANDRA-16047
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16047
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Wei Deng
>Priority: Urgent
> Attachments: incremental_backup_hardlink_exception.jpg, 
> incremental_backup_hardlink_exception1.jpg
>
>
> It seems that there is a race condition in creating hard link if incremental 
> backup is turned on.
> The following screenshot was captured in a production cluster running 
> Cassandra 3.0.15 after turning on incremental backup. When this 
> {{NoSuchFileException}} happens, due to the {{FSWriteError}} and the default 
> disk failure policy, the JVM will be shutdown, so it's a pretty critical bug.
>  !incremental_backup_hardlink_exception.jpg!
> Due to the risk of causing production database downtime (if similar issue 
> happens on multiple nodes in a short time frame), and same exception causing 
> JVM shutdown multiple times already, incremental backup had to be turned off 
> for now, but this is not an ideal situation.
> !incremental_backup_hardlink_exception1.jpg!
> The deployment is on a public cloud environment with EBS-like disks that are 
> backed by SSD with decent latency, throughput and IOPS, so it is hard to 
> think the culprit being in the OS and IO layer. Based on the second 
> screenshot above, this is a low flush traffic {{system.size_estimates}} 
> table, so compaction of the source SSTable doesn't seem to be at play here.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-16047) Potential race condition in creating hard link when incremental backup is turned on

2020-08-13 Thread Wei Deng (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-16047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-16047:
-
Description: 
It seems that there is a race condition in creating hard link if incremental 
backup is turned on.

The following screenshot was captured in a production cluster running Cassandra 
3.0.15 after turning on incremental backup. When this {{NoSuchFileException}} 
happens, due to the {{FSWriteError}} and the default disk failure policy, the 
JVM will be shutdown, so it's a pretty critical bug.
 !incremental_backup_hardlink_exception.jpg!

Due to the risk of causing production database downtime (if similar issue 
happens on multiple nodes in a short time frame), and similar issue causing JVM 
shutdown multiple times already, incremental backup had to be turned off for 
now, but this is not an ideal situation.

!incremental_backup_hardlink_exception1.jpg!

The deployment is on a public cloud environment with EBS-like disks that are 
backed by SSD with decent latency, throughput and IOPS, so it is hard to think 
the culprit being in the OS and IO layer. Based on the second screenshot above, 
this is a low flush traffic {{system.size_estimates}} table, so compaction of 
the source SSTable doesn't seem to be at play here.

  was:
It seems that there is a race condition in creating hard link if incremental 
backup is turned on.

The following screenshot was captured in a production cluster running Cassandra 
3.0.15 after turning on incremental backup. When this {{NoSuchFileException}} 
happens, due to the {{FSWriteError}} and the default disk failure policy, the 
JVM will be shutdown, so it's a pretty critical bug.
 !incremental_backup_hardlink_exception.jpg! 

 Due to the risk of causing production database downtime (if similar issue 
happens on multiple nodes in a short time frame), incremental backup had to be 
turned off for now, but this is not an ideal situation.

!incremental_backup_hardlink_exception1.jpg!

The deployment is on a public cloud environment with EBS-like disks that are 
backed by SSD with decent latency, throughput and IOPS, so it is hard to think 
the culprit being in the OS and IO layer. Based on the second screenshot above, 
this is a low flush traffic {{system.size_estimates}} table, so compaction of 
the source SSTable doesn't seem to be at play here.


> Potential race condition in creating hard link when incremental backup is 
> turned on
> ---
>
> Key: CASSANDRA-16047
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16047
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Wei Deng
>Priority: Urgent
> Attachments: incremental_backup_hardlink_exception.jpg, 
> incremental_backup_hardlink_exception1.jpg
>
>
> It seems that there is a race condition in creating hard link if incremental 
> backup is turned on.
> The following screenshot was captured in a production cluster running 
> Cassandra 3.0.15 after turning on incremental backup. When this 
> {{NoSuchFileException}} happens, due to the {{FSWriteError}} and the default 
> disk failure policy, the JVM will be shutdown, so it's a pretty critical bug.
>  !incremental_backup_hardlink_exception.jpg!
> Due to the risk of causing production database downtime (if similar issue 
> happens on multiple nodes in a short time frame), and similar issue causing 
> JVM shutdown multiple times already, incremental backup had to be turned off 
> for now, but this is not an ideal situation.
> !incremental_backup_hardlink_exception1.jpg!
> The deployment is on a public cloud environment with EBS-like disks that are 
> backed by SSD with decent latency, throughput and IOPS, so it is hard to 
> think the culprit being in the OS and IO layer. Based on the second 
> screenshot above, this is a low flush traffic {{system.size_estimates}} 
> table, so compaction of the source SSTable doesn't seem to be at play here.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-16047) Potential race condition in creating hard link when incremental backup is turned on

2020-08-13 Thread Wei Deng (Jira)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-16047?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-16047:
-
Bug Category: Parent values: Availability(12983)Level 1 values: Process 
Crash(12992)
Severity: Critical

> Potential race condition in creating hard link when incremental backup is 
> turned on
> ---
>
> Key: CASSANDRA-16047
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16047
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Wei Deng
>Priority: Urgent
> Attachments: incremental_backup_hardlink_exception.jpg, 
> incremental_backup_hardlink_exception1.jpg
>
>
> It seems that there is a race condition in creating hard link if incremental 
> backup is turned on.
> The following screenshot was captured in a production cluster running 
> Cassandra 3.0.15 after turning on incremental backup. When this 
> {{NoSuchFileException}} happens, due to the {{FSWriteError}} and the default 
> disk failure policy, the JVM will be shutdown, so it's a pretty critical bug.
>  !incremental_backup_hardlink_exception.jpg! 
>  Due to the risk of causing production database downtime (if similar issue 
> happens on multiple nodes in a short time frame), incremental backup had to 
> be turned off for now, but this is not an ideal situation.
> !incremental_backup_hardlink_exception1.jpg!
> The deployment is on a public cloud environment with EBS-like disks that are 
> backed by SSD with decent latency, throughput and IOPS, so it is hard to 
> think the culprit being in the OS and IO layer. Based on the second 
> screenshot above, this is a low flush traffic {{system.size_estimates}} 
> table, so compaction of the source SSTable doesn't seem to be at play here.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Created] (CASSANDRA-16047) Potential race condition in creating hard link when incremental backup is turned on

2020-08-13 Thread Wei Deng (Jira)
Wei Deng created CASSANDRA-16047:


 Summary: Potential race condition in creating hard link when 
incremental backup is turned on
 Key: CASSANDRA-16047
 URL: https://issues.apache.org/jira/browse/CASSANDRA-16047
 Project: Cassandra
  Issue Type: Bug
  Components: Local/SSTable
Reporter: Wei Deng
 Attachments: incremental_backup_hardlink_exception.jpg, 
incremental_backup_hardlink_exception1.jpg

It seems that there is a race condition in creating hard link if incremental 
backup is turned on.

The following screenshot was captured in a production cluster running Cassandra 
3.0.15 after turning on incremental backup. When this {{NoSuchFileException}} 
happens, due to the {{FSWriteError}} and the default disk failure policy, the 
JVM will be shutdown, so it's a pretty critical bug.
 !incremental_backup_hardlink_exception.jpg! 

 Due to the risk of causing production database downtime (if similar issue 
happens on multiple nodes in a short time frame), incremental backup had to be 
turned off for now, but this is not an ideal situation.

!incremental_backup_hardlink_exception1.jpg!

The deployment is on a public cloud environment with EBS-like disks that are 
backed by SSD with decent latency, throughput and IOPS, so it is hard to think 
the culprit being in the OS and IO layer. Based on the second screenshot above, 
this is a low flush traffic {{system.size_estimates}} table, so compaction of 
the source SSTable doesn't seem to be at play here.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-10789) Allow DBAs to kill individual client sessions from certain IP(s) and temporarily block subsequent connections without bouncing JVM

2018-05-08 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-10789:
-
Description: 
In production, there could be hundreds of clients connected to a Cassandra 
cluster (maybe even from different applications), and if they use DataStax Java 
Driver, each client will establish at least one TCP connection to a Cassandra 
server (see https://datastax.github.io/java-driver/2.1.9/features/pooling/). 
This is all normal and at any given time, you can indeed see hundreds of 
ESTABLISHED connections to port 9042 on a C* server (from netstat -na). The 
problem is that sometimes when a C* cluster is under heavy load, when the DBA 
identifies some client session that sends abusive amount of traffic to the C* 
server and would like to stop it, they would like a lightweight approach rather 
than shutting down the JVM or rolling restart the whole cluster to kill all 
hundreds of connections in order to kill a single client session. If the DBA 
had root privilege, they would have been able to do something at the OS network 
level to achieve the same goal but oftentimes enterprise DBA role is separate 
from OS sysadmin role, so the DBAs usually don't have that privilege.

This is especially helpful when you have a multi-tenant C* cluster and you want 
to have the impact for handling such client to be minimal to the other 
applications. This feature (killing individual session) seems to be a common 
feature in other databases (regardless of whether the client has some reconnect 
logic or not). It could be implemented as a JMX MBean method and exposed 
through nodetool to the DBAs.

Note due to CQL driver's automated reconnection, simply killing the currently 
connected client session will not work well, so the JMX parameter should be an 
IP address or a list of IP addresses, so that the Cassandra server can 
terminate existing connection with that IP, and block future connection 
attempts from that IP for the remaining time until the JVM is restarted.

  was:
In production, there could be hundreds of clients connected to a Cassandra 
cluster (maybe even from different applications), and if they use DataStax Java 
Driver, each client will establish at least one TCP connection to a Cassandra 
server (see https://datastax.github.io/java-driver/2.1.9/features/pooling/). 
This is all normal and at any given time, you can indeed see hundreds of 
ESTABLISHED connections to port 9042 on a C* server (from netstat -na). The 
problem is that sometimes when a C* cluster is under heavy load, when the DBA 
identifies some client session that sends abusive amount of traffic to the C* 
server and would like to stop it, they would like a lightweight approach rather 
than shutting down the JVM or rolling restart the whole cluster to kill all 
hundreds of connections in order to kill a single client session. If the DBA 
had root privilege, they would have been able to do something at the OS network 
level to achieve the same goal but oftentimes enterprise DBA role is separate 
from OS sysadmin role, so the DBAs usually don't have that privilege.

This is especially helpful when you have a multi-tenant C* cluster and you want 
to have the impact for handling such client to be minimal to the other 
applications. This feature (killing individual session) seems to be a common 
feature in other databases (regardless of whether the client has some reconnect 
logic or not). It could be implemented as a JMX MBean method and exposed 
through nodetool to the DBAs.


> Allow DBAs to kill individual client sessions from certain IP(s) and 
> temporarily block subsequent connections without bouncing JVM
> --
>
> Key: CASSANDRA-10789
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10789
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
>Reporter: Wei Deng
>Assignee: Damien Stevenson
>Priority: Major
> Fix For: 4.x
>
> Attachments: 10789-trunk-dtest.txt, 10789-trunk.txt
>
>
> In production, there could be hundreds of clients connected to a Cassandra 
> cluster (maybe even from different applications), and if they use DataStax 
> Java Driver, each client will establish at least one TCP connection to a 
> Cassandra server (see 
> https://datastax.github.io/java-driver/2.1.9/features/pooling/). This is all 
> normal and at any given time, you can indeed see hundreds of ESTABLISHED 
> connections to port 9042 on a C* server (from netstat -na). The problem is 
> that sometimes when a C* cluster is under heavy load, when the DBA identifies 
> some client session that sends abusive amount of traffic to the C* server and 
> would like to stop it, they would 

[jira] [Updated] (CASSANDRA-10789) Allow DBAs to kill individual client sessions from certain IP(s) and temporarily block subsequent connections without bouncing JVM

2018-05-08 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-10789:
-
Summary: Allow DBAs to kill individual client sessions from certain IP(s) 
and temporarily block subsequent connections without bouncing JVM  (was: Allow 
DBAs to kill individual client sessions without bouncing JVM)

> Allow DBAs to kill individual client sessions from certain IP(s) and 
> temporarily block subsequent connections without bouncing JVM
> --
>
> Key: CASSANDRA-10789
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10789
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
>Reporter: Wei Deng
>Assignee: Damien Stevenson
>Priority: Major
> Fix For: 4.x
>
> Attachments: 10789-trunk-dtest.txt, 10789-trunk.txt
>
>
> In production, there could be hundreds of clients connected to a Cassandra 
> cluster (maybe even from different applications), and if they use DataStax 
> Java Driver, each client will establish at least one TCP connection to a 
> Cassandra server (see 
> https://datastax.github.io/java-driver/2.1.9/features/pooling/). This is all 
> normal and at any given time, you can indeed see hundreds of ESTABLISHED 
> connections to port 9042 on a C* server (from netstat -na). The problem is 
> that sometimes when a C* cluster is under heavy load, when the DBA identifies 
> some client session that sends abusive amount of traffic to the C* server and 
> would like to stop it, they would like a lightweight approach rather than 
> shutting down the JVM or rolling restart the whole cluster to kill all 
> hundreds of connections in order to kill a single client session. If the DBA 
> had root privilege, they would have been able to do something at the OS 
> network level to achieve the same goal but oftentimes enterprise DBA role is 
> separate from OS sysadmin role, so the DBAs usually don't have that privilege.
> This is especially helpful when you have a multi-tenant C* cluster and you 
> want to have the impact for handling such client to be minimal to the other 
> applications. This feature (killing individual session) seems to be a common 
> feature in other databases (regardless of whether the client has some 
> reconnect logic or not). It could be implemented as a JMX MBean method and 
> exposed through nodetool to the DBAs.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-12464) Investigate the potential improvement of parallelism on higher level compactions in LCS

2018-04-30 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16458922#comment-16458922
 ] 

Wei Deng commented on CASSANDRA-12464:
--

Thanks. Updated the JIRA description.

> Investigate the potential improvement of parallelism on higher level 
> compactions in LCS
> ---
>
> Key: CASSANDRA-12464
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12464
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>Priority: Major
>  Labels: lcs, lhf, performance
>
> According to LevelDB's design doc 
> [here|https://github.com/google/leveldb/blob/master/doc/impl.md#compactions], 
> "A compaction merges the contents of the picked files to produce a sequence 
> of level-(L+1) files", it will "switch to producing a new level-(L+1) file 
> after the current output file has reached the target file size" (in our case 
> 160MB), it will also "switch to a new output file when the key range of the 
> current output file has grown enough to overlap more than ten level-(L+2) 
> files". This is to ensure "that a later compaction of a level-(L+1) file will 
> not pick up too much data from level-(L+2)."
> Our current code in LeveledCompactionStrategy doesn't implement this last 
> rule, but we might be able to quickly implement it and see how much a 
> compaction throughput improvement it can deliver. Potentially we can create a 
> scenario where a number of large L0 SSTables are present (e.g. 200GB after 
> switching from STCS) and let it to create thousands of L1 SSTables overflow, 
> and see how fast LCS can digest this much data from L1 and properly 
> upper-level them to completion.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-12464) Investigate the potential improvement of parallelism on higher level compactions in LCS

2018-04-30 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12464:
-
Description: 
According to LevelDB's design doc 
[here|https://github.com/google/leveldb/blob/master/doc/impl.md#compactions], 
"A compaction merges the contents of the picked files to produce a sequence of 
level-(L+1) files", it will "switch to producing a new level-(L+1) file after 
the current output file has reached the target file size" (in our case 160MB), 
it will also "switch to a new output file when the key range of the current 
output file has grown enough to overlap more than ten level-(L+2) files". This 
is to ensure "that a later compaction of a level-(L+1) file will not pick up 
too much data from level-(L+2)."

Our current code in LeveledCompactionStrategy doesn't implement this last rule, 
but we might be able to quickly implement it and see how much a compaction 
throughput improvement it can deliver. Potentially we can create a scenario 
where a number of large L0 SSTables are present (e.g. 200GB after switching 
from STCS) and let it to create thousands of L1 SSTables overflow, and see how 
fast LCS can digest this much data from L1 and properly upper-level them to 
completion.

  was:
According to LevelDB's design doc 
[here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
"A compaction merges the contents of the picked files to produce a sequence of 
level-(L+1) files", it will "switch to producing a new level-(L+1) file after 
the current output file has reached the target file size" (in our case 160MB), 
it will also "switch to a new output file when the key range of the current 
output file has grown enough to overlap more than ten level-(L+2) files". This 
is to ensure "that a later compaction of a level-(L+1) file will not pick up 
too much data from level-(L+2)."

Our current code in LeveledCompactionStrategy doesn't implement this last rule, 
but we might be able to quickly implement it and see how much a compaction 
throughput improvement it can deliver. Potentially we can create a scenario 
where a number of large L0 SSTables are present (e.g. 200GB after switching 
from STCS) and let it to create thousands of L1 SSTables overflow, and see how 
fast LCS can digest this much data from L1 and properly upper-level them to 
completion.


> Investigate the potential improvement of parallelism on higher level 
> compactions in LCS
> ---
>
> Key: CASSANDRA-12464
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12464
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>Priority: Major
>  Labels: lcs, lhf, performance
>
> According to LevelDB's design doc 
> [here|https://github.com/google/leveldb/blob/master/doc/impl.md#compactions], 
> "A compaction merges the contents of the picked files to produce a sequence 
> of level-(L+1) files", it will "switch to producing a new level-(L+1) file 
> after the current output file has reached the target file size" (in our case 
> 160MB), it will also "switch to a new output file when the key range of the 
> current output file has grown enough to overlap more than ten level-(L+2) 
> files". This is to ensure "that a later compaction of a level-(L+1) file will 
> not pick up too much data from level-(L+2)."
> Our current code in LeveledCompactionStrategy doesn't implement this last 
> rule, but we might be able to quickly implement it and see how much a 
> compaction throughput improvement it can deliver. Potentially we can create a 
> scenario where a number of large L0 SSTables are present (e.g. 200GB after 
> switching from STCS) and let it to create thousands of L1 SSTables overflow, 
> and see how fast LCS can digest this much data from L1 and properly 
> upper-level them to completion.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13294) Possible data loss on upgrade 2.1 - 3.0

2017-03-03 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-13294?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895295#comment-15895295
 ] 

Wei Deng commented on CASSANDRA-13294:
--

If this problem is caused by the file name change, would we need to port to 2.2 
as well, because the file name change was introduced in 2.2?

> Possible data loss on upgrade 2.1 - 3.0
> ---
>
> Key: CASSANDRA-13294
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13294
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Blocker
> Fix For: 3.0.x
>
>
> After finishing a compaction we delete the compacted away files. This is done 
> [here|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/db/lifecycle/LogFile.java#L328-L337]
>  which uses 
> [this|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java#L265-L271]
>  to get the files - we get all files starting with {{absoluteFilePath}}. 
> Absolute file path is generated 
> [here|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/io/sstable/Descriptor.java#L142-L153].
>  For 3.0 version files the filename looks like this: 
> {{/blabla/keyspace1/standard1-bdb031c0ff7b11e6940fdd0479dd8912/mc-1332-big}} 
> but for 2.1 version files, they look like this: 
> {{/blabla/keyspace1/standard1-bdb031c0ff7b11e6940fdd0479dd8912/keyspace1-standard1-ka-2}}.
> The problem is then that if we were to finish a compaction including the 
> legacy file, we would actually delete all legacy files having a generation 
> starting with '2'



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (CASSANDRA-13162) Batchlog replay is throttled during bootstrap, creating conditions for incorrect query results on materialized views

2017-01-27 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13162?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-13162:
-
Description: 
I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
defined (one base table and two MVs on that base table). The data volume is not 
very high per node (about 80GB of data per node total, and that particular base 
table has about 25GB of data uncompressed with one MV taking 18GB compressed 
and the other MV taking 3GB), and the cluster is using decent hardware (EC2 
C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB gp2 EBS volumes). 

This is originally a 9-node cluster. It appears that after adding 3 more nodes 
to the DC, the system.batches table accumulated a lot of data on the 3 new 
nodes (each having around 20GB under system.batches directory), and in the 
subsequent week the batchlog on the 3 new nodes got slowly replayed back to the 
rest of the nodes in the cluster. The bottleneck seems to be the throttling 
defined in this cassandra.yaml setting: batchlog_replay_throttle_in_kb, which 
by default is set to 1MB/s.

Given that it is taking almost a week (and still hasn't finished) for the 
batchlog (from MV) to be replayed after the boostrap finishes, it seems only 
reasonable to unthrottle (or at least give it a much higher throttle rate) 
during the initial bootstrap, and hence I'd consider this a bug for our current 
MV implementation.

Also as far as I understand, the bootstrap logic won't wait for the backlogged 
batchlog to be fully replayed before changing the new bootstrapping node to 
"UN" state, and if batchlog for the MVs got stuck in this state for a long 
time, we basically will get wrong answers on the MVs during that whole duration 
(until batchlog is fully played to the cluster), which adds even more 
criticality to this bug.

  was:
I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
defined (one base table and two MVs on that base table). The data volume is not 
very high per node (about 80GB of data per node total, and that particular base 
table has about 25GB of data uncompressed with one MV taking 18GB compressed 
and the other MV taking 3GB), and the cluster is using decent hardware (EC2 
C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB gp2 EBS volumes). 

This is originally a 9-node cluster. It appears that after adding 3 more nodes 
to the DC, the system.batches table accumulated a lot of data on the 3 new 
nodes, and in the subsequent week the batchlog on the 3 new nodes got slowly 
replayed back to the rest of the nodes in the cluster. The bottleneck seems to 
be the throttling defined in this cassandra.yaml setting: 
batchlog_replay_throttle_in_kb, which by default is set to 1MB/s.

Given that it is taking almost a week (and still hasn't finished) for the 
batchlog (from MV) to be replayed after the boostrap finishes, it seems only 
reasonable to unthrottle (or at least give it a much higher throttle rate) 
during the initial bootstrap, and hence I'd consider this a bug for our current 
MV implementation.

Also as far as I understand, the bootstrap logic won't wait for the backlogged 
batchlog to be fully replayed before changing the new bootstrapping node to 
"UN" state, and if batchlog for the MVs got stuck in this state for a long 
time, we basically will get wrong answers on the MVs during that whole duration 
(until batchlog is fully played to the cluster), which adds even more 
criticality to this bug.


> Batchlog replay is throttled during bootstrap, creating conditions for 
> incorrect query results on materialized views
> 
>
> Key: CASSANDRA-13162
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13162
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>Priority: Critical
>  Labels: bootstrap, materializedviews
>
> I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
> defined (one base table and two MVs on that base table). The data volume is 
> not very high per node (about 80GB of data per node total, and that 
> particular base table has about 25GB of data uncompressed with one MV taking 
> 18GB compressed and the other MV taking 3GB), and the cluster is using decent 
> hardware (EC2 C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB 
> gp2 EBS volumes). 
> This is originally a 9-node cluster. It appears that after adding 3 more 
> nodes to the DC, the system.batches table accumulated a lot of data on the 3 
> new nodes (each having around 20GB under system.batches directory), and in 
> the subsequent week the batchlog on the 3 new nodes got slowly replayed back 
> to the rest of the nodes in the cluster. The bottleneck seems to 

[jira] [Updated] (CASSANDRA-13162) Batchlog replay is throttled during bootstrap, creating conditions for incorrect query results on materialized views

2017-01-27 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13162?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-13162:
-
Labels: bootstrap materializedviews  (was: )

> Batchlog replay is throttled during bootstrap, creating conditions for 
> incorrect query results on materialized views
> 
>
> Key: CASSANDRA-13162
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13162
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>Priority: Critical
>  Labels: bootstrap, materializedviews
>
> I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
> defined (one base table and two MVs on that base table). The data volume is 
> not very high per node (about 80GB of data per node total, and that 
> particular base table has about 25GB of data uncompressed with one MV taking 
> 18GB compressed and the other MV taking 3GB), and the cluster is using decent 
> hardware (EC2 C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB 
> gp2 EBS volumes). 
> This is originally a 9-node cluster. It appears that after adding 3 more 
> nodes to the DC, the system.batches table accumulated a lot of data on the 3 
> new nodes, and in the subsequent week the batchlog on the 3 new nodes got 
> slowly replayed back to the rest of the nodes in the cluster. The bottleneck 
> seems to be the throttling defined in this cassandra.yaml setting: 
> batchlog_replay_throttle_in_kb, which by default is set to 1MB/s.
> Given that it is taking almost a week (and still hasn't finished) for the 
> batchlog (from MV) to be replayed after the boostrap finishes, it seems only 
> reasonable to unthrottle (or at least give it a much higher throttle rate) 
> during the initial bootstrap, and hence I'd consider this a bug for our 
> current MV implementation.
> Also as far as I understand, the bootstrap logic won't wait for the 
> backlogged batchlog to be fully replayed before changing the new 
> bootstrapping node to "UN" state, and if batchlog for the MVs got stuck in 
> this state for a long time, we basically will get wrong answers on the MVs 
> during that whole duration (until batchlog is fully played to the cluster), 
> which adds even more criticality to this bug.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-13162) Batchlog replay is throttled during bootstrap, creating conditions for incorrect query results on materialized views

2017-01-27 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13162?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-13162:
-
Priority: Critical  (was: Major)

> Batchlog replay is throttled during bootstrap, creating conditions for 
> incorrect query results on materialized views
> 
>
> Key: CASSANDRA-13162
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13162
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>Priority: Critical
>
> I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
> defined (one base table and two MVs on that base table). The data volume is 
> not very high per node (about 80GB of data per node total, and that 
> particular base table has about 25GB of data uncompressed with one MV taking 
> 18GB compressed and the other MV taking 3GB), and the cluster is using decent 
> hardware (EC2 C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB 
> gp2 EBS volumes). 
> This is originally a 9-node cluster. It appears that after adding 3 more 
> nodes to the DC, the system.batches table accumulated a lot of data on the 3 
> new nodes, and in the subsequent week the batchlog on the 3 new nodes got 
> slowly replayed back to the rest of the nodes in the cluster. The bottleneck 
> seems to be the throttling defined in this cassandra.yaml setting: 
> batchlog_replay_throttle_in_kb, which by default is set to 1MB/s.
> Given that it is taking almost a week (and still hasn't finished) for the 
> batchlog (from MV) to be replayed after the boostrap finishes, it seems only 
> reasonable to unthrottle (or at least give it a much higher throttle rate) 
> during the initial bootstrap, and hence I'd consider this a bug for our 
> current MV implementation.
> Also as far as I understand, the bootstrap logic won't wait for the 
> backlogged batchlog to be fully replayed before changing the new 
> bootstrapping node to "UN" state, and if batchlog for the MVs got stuck in 
> this state for a long time, we basically will get wrong answers on the MVs 
> during that whole duration (until batchlog is fully played to the cluster), 
> which adds even more criticality to this bug.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-13162) Batchlog replay is throttled during bootstrap

2017-01-27 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-13162:


 Summary: Batchlog replay is throttled during bootstrap
 Key: CASSANDRA-13162
 URL: https://issues.apache.org/jira/browse/CASSANDRA-13162
 Project: Cassandra
  Issue Type: Bug
Reporter: Wei Deng


I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
defined (one base table and two MVs on that base table). The data volume is not 
very high per node (about 80GB of data per node total, and that particular base 
table has about 25GB of data uncompressed with one MV taking 18GB compressed 
and the other MV taking 3GB), and the cluster is using decent hardware (EC2 
C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB gp2 EBS volumes). 

This is originally a 9-node cluster. It appears that after adding 3 more nodes 
to the DC, the system.batches table accumulated a lot of data on the 3 new 
nodes, and in the subsequent week the batchlog on the 3 new nodes got slowly 
replayed back to the rest of the nodes in the cluster. The bottleneck seems to 
be the throttling defined in this cassandra.yaml setting: 
batchlog_replay_throttle_in_kb, which by default is set to 1MB/s.

Given that it is taking almost a week (and still hasn't finished) for the 
batchlog (from MV) to be replayed after the boostrap finishes, it seems only 
reasonable to unthrottle (or at least give it a much higher throttle rate) 
during the initial bootstrap, and hence I'd consider this a bug for our current 
MV implementation.

Also as far as I understand, the bootstrap logic won't wait for the backlogged 
batchlog to be fully replayed before changing the new bootstrapping node to 
"UN" state, and if batchlog for the MVs got stuck in this state for a long 
time, we basically will get wrong answers on the MVs during that whole duration 
(until batchlog is fully played to the cluster), which adds even more 
criticality to this bug.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-13162) Batchlog replay is throttled during bootstrap, creating conditions for incorrect query results on materialized views

2017-01-27 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13162?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-13162:
-
Summary: Batchlog replay is throttled during bootstrap, creating conditions 
for incorrect query results on materialized views  (was: Batchlog replay is 
throttled during bootstrap)

> Batchlog replay is throttled during bootstrap, creating conditions for 
> incorrect query results on materialized views
> 
>
> Key: CASSANDRA-13162
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13162
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> I've tested this in a C* 3.0 cluster with a couple of Materialized Views 
> defined (one base table and two MVs on that base table). The data volume is 
> not very high per node (about 80GB of data per node total, and that 
> particular base table has about 25GB of data uncompressed with one MV taking 
> 18GB compressed and the other MV taking 3GB), and the cluster is using decent 
> hardware (EC2 C4.8XL with 18 cores + 60GB RAM + 18K IOPS RAID0 from two 3TB 
> gp2 EBS volumes). 
> This is originally a 9-node cluster. It appears that after adding 3 more 
> nodes to the DC, the system.batches table accumulated a lot of data on the 3 
> new nodes, and in the subsequent week the batchlog on the 3 new nodes got 
> slowly replayed back to the rest of the nodes in the cluster. The bottleneck 
> seems to be the throttling defined in this cassandra.yaml setting: 
> batchlog_replay_throttle_in_kb, which by default is set to 1MB/s.
> Given that it is taking almost a week (and still hasn't finished) for the 
> batchlog (from MV) to be replayed after the boostrap finishes, it seems only 
> reasonable to unthrottle (or at least give it a much higher throttle rate) 
> during the initial bootstrap, and hence I'd consider this a bug for our 
> current MV implementation.
> Also as far as I understand, the bootstrap logic won't wait for the 
> backlogged batchlog to be fully replayed before changing the new 
> bootstrapping node to "UN" state, and if batchlog for the MVs got stuck in 
> this state for a long time, we basically will get wrong answers on the MVs 
> during that whole duration (until batchlog is fully played to the cluster), 
> which adds even more criticality to this bug.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-13107) Remove -XX:ThreadPriorityPolicy=42 jvm flag

2017-01-06 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-13107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15805106#comment-15805106
 ] 

Wei Deng commented on CASSANDRA-13107:
--

Do we need to create a related JIRA to track the need to find another 
workaround/solution so that non-root C* process can still lower priority of 
some threads?

> Remove -XX:ThreadPriorityPolicy=42 jvm flag
> ---
>
> Key: CASSANDRA-13107
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13107
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Paulo Motta
>Priority: Minor
>
> CASSANDRA-1181 added {{-XX:ThreadPriorityPolicy=42}} jvm flag to support 
> setting native thread priority without root (based on the workaround 
> described on 
> http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workaround.html).
> On Java9 this wokaround will not longer be possible due to [JEP 
> 245|https://bugs.openjdk.java.net/browse/JDK-8059557] so this flag should be 
> removed.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12182) redundant StatusLogger print out when both dropped message and long GC event happen

2016-10-12 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15570188#comment-15570188
 ] 

Wei Deng commented on CASSANDRA-12182:
--

[~KurtG] Personally I don't have much issue with the current logging level for 
StatusLogger. When the node is suffering, either because it's dropping 
non-TRACE messages, or because it's exceeding the 1sec gc_warn_threshold_in_ms 
threshold, I'd like to see StatusLogger to give me more information for 
post-mortem analysis and I don't want to change to DEBUG level to see it. Note 
"post-mortem" is the key here as you won't know when this will happen and if 
you have to switch to DEBUG level to see the message likely it will be too late.

bq. I think replacing the log messages with StatusLogger is busy would somewhat 
defeat the purpose.
Can you elaborate why you think avoiding duplicate StatusLogger printing 
"defeat the purpose"? The StatusLogger usually only takes 100-200ms to finish 
printing its state. If at the time StatusLogger is printing, another 
StatusLogger gets triggered and print again, it mostly just adds duplicate 
information which makes log messages crowded without adding any more useful 
insight.

> redundant StatusLogger print out when both dropped message and long GC event 
> happen
> ---
>
> Key: CASSANDRA-12182
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12182
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>Priority: Minor
>  Labels: lhf
>
> I was stress testing a C* 3.0 environment and it appears that when the CPU is 
> running low, HINT and MUTATION messages will start to get dropped, and the GC 
> thread can also get some really long-running GC, and I'd get some redundant 
> log entries in system.log like the following:
> {noformat}
> WARN  [Service Thread] 2016-07-12 22:48:45,748  GCInspector.java:282 - G1 
> Young Generation GC in 522ms.  G1 Eden Space: 68157440 -> 0; G1 Old Gen: 
> 3376113224 -> 3468387912; G1 Survivor Space: 24117248 -> 0; 
> INFO  [Service Thread] 2016-07-12 22:48:45,763  StatusLogger.java:52 - Pool 
> NameActive   Pending  Completed   Blocked  All Time 
> Blocked
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,775  MessagingService.java:983 - 
> MUTATION messages were dropped in last 5000 ms: 419 for internal timeout and 
> 0 for cross node timeout
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,776  MessagingService.java:983 - 
> HINT messages were dropped in last 5000 ms: 89 for internal timeout and 0 for 
> cross node timeout
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,776  StatusLogger.java:52 - Pool 
> NameActive   Pending  Completed   Blocked  All Time 
> Blocked
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,798  StatusLogger.java:56 - 
> MutationStage32  4194   32997234 0
>  0
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,798  StatusLogger.java:56 - 
> ViewMutationStage 0 0  0 0
>  0
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,799  StatusLogger.java:56 - 
> ReadStage 0 0940 0
>  0
> INFO  [Service Thread] 2016-07-12 22:48:45,800  StatusLogger.java:56 - 
> MutationStage32  4363   32997333 0
>  0
> INFO  [Service Thread] 2016-07-12 22:48:45,801  StatusLogger.java:56 - 
> ViewMutationStage 0 0  0 0
>  0
> INFO  [Service Thread] 2016-07-12 22:48:45,801  StatusLogger.java:56 - 
> ReadStage 0 0940 0
>  0
> INFO  [Service Thread] 2016-07-12 22:48:45,802  StatusLogger.java:56 - 
> RequestResponseStage  0 0   11094437 0
>  0
> INFO  [Service Thread] 2016-07-12 22:48:45,802  StatusLogger.java:56 - 
> ReadRepairStage   0 0  5 0
>  0
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,803  StatusLogger.java:56 - 
> RequestResponseStage  4 0   11094509 0
>  0
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,807  StatusLogger.java:56 - 
> ReadRepairStage   0 0  5 0
>  0
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,808  StatusLogger.java:56 - 
> CounterMutationStage  0 0  0 0
>  0
> INFO  [ScheduledTasks:1] 2016-07-12 22:48:45,809  StatusLogger.java:56 - 
> MiscStage 0 0  0 0
>  0
> INFO  

[jira] [Updated] (CASSANDRA-12764) Compaction performance issues with many sstables, during transaction commit phase

2016-10-10 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12764?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12764:
-
Labels: lcs  (was: )

> Compaction performance issues with many sstables, during transaction commit 
> phase
> -
>
> Key: CASSANDRA-12764
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12764
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Tom van der Woerdt
>  Labels: lcs
>
> An issue with a script flooded my cluster with sstables. There is now a table 
> with 100k sstables, all on the order of KBytes, and it's taking a long time 
> (ETA 20 days) to compact, even though the table is only ~30GB.
> Stack trace :
> {noformat}
> "CompactionExecutor:308" #7541 daemon prio=1 os_prio=4 tid=0x7fa22af35400 
> nid=0x41eb runnable [0x7fdbea48d000]
>java.lang.Thread.State: RUNNABLE
>   at java.util.TimSort.countRunAndMakeAscending(TimSort.java:360)
>   at java.util.TimSort.sort(TimSort.java:220)
>   at java.util.Arrays.sort(Arrays.java:1438)
>   at com.google.common.collect.Ordering.sortedCopy(Ordering.java:817)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:209)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:211)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:210)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:210)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:210)
>   at 
> org.apache.cassandra.utils.IntervalTree$IntervalNode.(IntervalTree.java:210)
>   at org.apache.cassandra.utils.IntervalTree.(IntervalTree.java:50)
>   at 
> org.apache.cassandra.db.lifecycle.SSTableIntervalTree.(SSTableIntervalTree.java:40)
>   at 
> org.apache.cassandra.db.lifecycle.SSTableIntervalTree.build(SSTableIntervalTree.java:50)
>   at org.apache.cassandra.db.lifecycle.View$4.apply(View.java:288)
>   at org.apache.cassandra.db.lifecycle.View$4.apply(View.java:283)
>   at 
> com.google.common.base.Functions$FunctionComposition.apply(Functions.java:216)
>   at org.apache.cassandra.db.lifecycle.Tracker.apply(Tracker.java:128)
>   at org.apache.cassandra.db.lifecycle.Tracker.apply(Tracker.java:101)
>   at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkpoint(LifecycleTransaction.java:307)
>   at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.checkpoint(LifecycleTransaction.java:288)
>   at 
> org.apache.cassandra.io.sstable.SSTableRewriter.doPrepare(SSTableRewriter.java:368)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>   at 
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.doPrepare(CompactionAwareWriter.java:84)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish(Transactional.java:184)
>   at 
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.finish(CompactionAwareWriter.java:94)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:194)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:78)
>   at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
>   at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:263)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>   at java.lang.Thread.run(Thread.java:745)
> {noformat}
> IntervalTree shows 

[jira] [Updated] (CASSANDRA-12763) Compaction performance issues when a table has a lot of sstables

2016-10-10 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12763:
-
Labels: lcs  (was: )

> Compaction performance issues when a table has a lot of sstables
> 
>
> Key: CASSANDRA-12763
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12763
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Tom van der Woerdt
>  Labels: lcs
>
> An issue with a script flooded my cluster with sstables. There is now a table 
> with 100k sstables, all on the order of KBytes, and it's taking a long time 
> (ETA 20 days) to compact, even though the table is only ~30GB.
> Stack trace :
> {noformat}
> "CompactionExecutor:269" #7536 daemon prio=1 os_prio=4 tid=0x7f4acd40fc00 
> nid=0x14f8 runnable [0x7f4798436000]
>java.lang.Thread.State: RUNNABLE
>   at java.io.UnixFileSystem.list(Native Method)
>   at java.io.File.list(File.java:1122)
>   at java.io.File.listFiles(File.java:1248)
>   at 
> org.apache.cassandra.db.lifecycle.LogRecord.getExistingFiles(LogRecord.java:268)
>   at org.apache.cassandra.db.lifecycle.LogRecord.make(LogRecord.java:150)
>   at 
> org.apache.cassandra.db.lifecycle.LogFile.makeRecord(LogFile.java:293)
>   at org.apache.cassandra.db.lifecycle.LogFile.add(LogFile.java:283)
>   at 
> org.apache.cassandra.db.lifecycle.LogTransaction.obsoleted(LogTransaction.java:158)
>   at 
> org.apache.cassandra.db.lifecycle.Helpers.prepareForObsoletion(Helpers.java:134)
>   at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.doPrepare(LifecycleTransaction.java:193)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>   at 
> org.apache.cassandra.io.sstable.SSTableRewriter.doPrepare(SSTableRewriter.java:376)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>   at 
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.doPrepare(CompactionAwareWriter.java:84)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.prepareToCommit(Transactional.java:173)
>   at 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish(Transactional.java:184)
>   at 
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.finish(CompactionAwareWriter.java:94)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:194)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:78)
>   at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
>   at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:263)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>   at java.lang.Thread.run(Thread.java:745)
> {noformat}
> listFiles is being called over and over, apparently scaling with the number 
> of files in the compaction.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12659) Query in reversed order brough back deleted data

2016-09-22 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12659?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15514545#comment-15514545
 ] 

Wei Deng commented on CASSANDRA-12659:
--

[~khuutan...@gmail.com] You can move SSTable from one cluster to another, 
assuming the 2nd cluster can replicate the token ownership of the previous 
cluster, and you also replicate the schema on the 2nd cluster. You probably 
don't want to use sstableloader in this case as it might change the sstables 
when they're landed on the destination.

> Query in reversed order brough back deleted data
> 
>
> Key: CASSANDRA-12659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12659
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 3.0.5, 6 nodes cluster
>Reporter: Tai Khuu Tan
>
> We have and issues with our Cassandra 3.0.5. After we deleted a large amount 
> of data in the multiple partition keys. Query those partition keys with 
> reversed order on a clustering key return the deleted data. I have checked 
> and there are no tombstones left. All of them are deleted. So I don't know 
> where or how can those deleted data still exist. Is there any other place 
> that Cassandra will read data when query in reverse order compare to normal 
> order ?
> the schema is very simple
> {noformat}
> CREATE TABLE table ( uid varchar, version timestamp, data1 varchar, data2 
> varchar, data3 varchar, data4 varchar, data5 varchar, PRIMARY KEY (uid, 
> version, data1 , data2 , data3 , data4 ) ) with compact storage;
> {noformat}
> Query are doing reverse order on column timestamp
> Ex:
> {noformat}
> select * from data where uid="uid1" order by version DESC
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12659) Query in reversed order brough back deleted data

2016-09-18 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12659?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15502203#comment-15502203
 ] 

Wei Deng commented on CASSANDRA-12659:
--

Did you preserve the SSTables (snapshot) when you were able to reproduce the 
problem? If yes, it will be useful to use "nodetool getendpoints" and "nodetool 
getsstables" to extract a number of SSTables that contains the partition in 
question and upload them here along with the schema (assuming it doesn't 
contain sensitive information). If you're no longer able to reproduce the 
problem, then there is no need to provide the SSTables.

Without a repro case, it will be hard for people to look into it. However, 
filing this JIRA right now is still valuable because if other people run into 
the same problem in a different occasion, they will have something to compare 
notes to determine if this is a real problem. If after a while nobody else runs 
into this issue and you are still not able to reproduce it, the JIRA could end 
up getting closed eventually.

> Query in reversed order brough back deleted data
> 
>
> Key: CASSANDRA-12659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12659
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 3.0.5, 6 nodes cluster
>Reporter: Tai Khuu Tan
>
> We have and issues with our Cassandra 3.0.5. After we deleted a large amount 
> of data in the multiple partition keys. Query those partition keys with 
> reversed order on a clustering key return the deleted data. I have checked 
> and there are no tombstones left. All of them are deleted. So I don't know 
> where or how can those deleted data still exist. Is there any other place 
> that Cassandra will read data when query in reverse order compare to normal 
> order ?
> the schema is very simple
> {noformat}
> CREATE TABLE table ( uid varchar, version timestamp, data1 varchar, data2 
> varchar, data3 varchar, data4 varchar, data5 varchar, PRIMARY KEY (uid, 
> version, data1 , data2 , data3 , data4 ) ) with compact storage;
> {noformat}
> Query are doing reverse order on column timestamp
> Ex:
> {noformat}
> select * from data where uid="uid1" order by version DESC
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12655) Incremental repair & compaction hang on random nodes

2016-09-17 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15499243#comment-15499243
 ] 

Wei Deng commented on CASSANDRA-12655:
--

BTW, in addition to the behavior that anti-compaction getting blocked by other 
regular compactions, you may have run into some other compaction hang issue 
with the regular compactions (especially when you say "nodetool 
compactionstats" also hang forever), as Marcus pointed out. When you run into 
this issue again, if you can always see CPU being completely idle with no CPU 
core working on any regular compaction threads while anti-compaction is still 
blocked by all pending regular compactions, then you've likely run into one of 
those compaction hang bugs in earlier version of 2.2.x.

Definitely moving to the latest 2.2.x version will help to avoid those known 
and fixed compaction hang problems and that should be a required first step. 
Then you will need to wait for the improvement in CASSANDRA-12200 to completely 
avoid trivial repair from being blocked by backlogged compactions. As 
CASSANDRA-12200 is an improvement instead of a bug and likely will not go into 
2.2, you likely will need to plan to cherry-pick the fix and back-port to your 
own 2.2 version, if you don't have plan to go to 3.x shortly.

> Incremental repair & compaction hang on random nodes
> 
>
> Key: CASSANDRA-12655
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12655
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: CentOS Linux release 7.1.1503 (Core)
> RAM - 64GB
> HEAP - 16GB
> Load on each node - ~5GB
> Cassandra Version - 2.2.5
>Reporter: Navjyot Nishant
>Priority: Blocker
>
> Hi We are setting up incremental repair on our 18 node cluster. Avg load on 
> each node is ~5GB. The repair run fine on couple of nodes and sudently get 
> stuck on random nodes. Upon checking the system.log of impacted node we dont 
> see much information.
> Following are the lines we see in system.log and its there from the point 
> repair is not making progress -
> {code}
> INFO  [CompactionExecutor:3490] 2016-09-16 11:14:44,236 
> CompactionManager.java:1221 - Anticompacting 
> [BigTableReader(path='/cassandra/data/gccatlgsvcks/message_backup-cab0485008ed11e5bfed452cdd54652d/la-30832-big-Data.db'),
>  
> BigTableReader(path='/cassandra/data/gccatlgsvcks/message_backup-cab0485008ed11e5bfed452cdd54652d/la-30811-big-Data.db')]
> INFO  [IndexSummaryManager:1] 2016-09-16 11:14:49,954 
> IndexSummaryRedistribution.java:74 - Redistributing index summaries
> INFO  [IndexSummaryManager:1] 2016-09-16 12:14:49,961 
> IndexSummaryRedistribution.java:74 - Redistributing index summaries
> {code}
> When we try to see pending compaction by executing {code}nodetool 
> compactionstats{code} it hangs as well and doesn't return anything. However 
> {code}nodetool tpstats{code} show active and pending compaction which never 
> come down and keep increasing. 
> {code}
> Pool NameActive   Pending  Completed   Blocked  All 
> time blocked
> MutationStage 0 0 221208 0
>  0
> ReadStage 0 01288839 0
>  0
> RequestResponseStage  0 0 104356 0
>  0
> ReadRepairStage   0 0 72 0
>  0
> CounterMutationStage  0 0  0 0
>  0
> HintedHandoff 0 0 46 0
>  0
> MiscStage 0 0  0 0
>  0
> CompactionExecutor866  68124 0
>  0
> MemtableReclaimMemory 0 0166 0
>  0
> PendingRangeCalculator0 0 38 0
>  0
> GossipStage   0 0 242455 0
>  0
> MigrationStage0 0  0 0
>  0
> MemtablePostFlush 0 0   3682 0
>  0
> ValidationExecutor0 0   2246 0
>  0
> Sampler   0 0  0 0
>  0
> MemtableFlushWriter   0 0166 0
>  0
> InternalResponseStage 0 0   8866 0
>  0
> AntiEntropyStage  0 0  15417 0
>  0
> Repair#7

[jira] [Commented] (CASSANDRA-12655) Incremental repair & compaction hang on random nodes

2016-09-17 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15499221#comment-15499221
 ] 

Wei Deng commented on CASSANDRA-12655:
--

This is duplicate to CASSANDRA-12200, which is currently being worked on by 
[~jjirsa]. It's not a bug per se, but the behavior indeed needs to be improved 
to avoid backlogged compactions (in your case, you can see you've got 66 
pending for CompactionExecutor) from blocking anti-compaction for a long time 
so that trivial repair cannot finish.

> Incremental repair & compaction hang on random nodes
> 
>
> Key: CASSANDRA-12655
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12655
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: CentOS Linux release 7.1.1503 (Core)
> RAM - 64GB
> HEAP - 16GB
> Load on each node - ~5GB
> Cassandra Version - 2.2.5
>Reporter: Navjyot Nishant
>Priority: Blocker
>
> Hi We are setting up incremental repair on our 18 node cluster. Avg load on 
> each node is ~5GB. The repair run fine on couple of nodes and sudently get 
> stuck on random nodes. Upon checking the system.log of impacted node we dont 
> see much information.
> Following are the lines we see in system.log and its there from the point 
> repair is not making progress -
> {code}
> INFO  [CompactionExecutor:3490] 2016-09-16 11:14:44,236 
> CompactionManager.java:1221 - Anticompacting 
> [BigTableReader(path='/cassandra/data/gccatlgsvcks/message_backup-cab0485008ed11e5bfed452cdd54652d/la-30832-big-Data.db'),
>  
> BigTableReader(path='/cassandra/data/gccatlgsvcks/message_backup-cab0485008ed11e5bfed452cdd54652d/la-30811-big-Data.db')]
> INFO  [IndexSummaryManager:1] 2016-09-16 11:14:49,954 
> IndexSummaryRedistribution.java:74 - Redistributing index summaries
> INFO  [IndexSummaryManager:1] 2016-09-16 12:14:49,961 
> IndexSummaryRedistribution.java:74 - Redistributing index summaries
> {code}
> When we try to see pending compaction by executing {code}nodetool 
> compactionstats{code} it hangs as well and doesn't return anything. However 
> {code}nodetool tpstats{code} show active and pending compaction which never 
> come down and keep increasing. 
> {code}
> Pool NameActive   Pending  Completed   Blocked  All 
> time blocked
> MutationStage 0 0 221208 0
>  0
> ReadStage 0 01288839 0
>  0
> RequestResponseStage  0 0 104356 0
>  0
> ReadRepairStage   0 0 72 0
>  0
> CounterMutationStage  0 0  0 0
>  0
> HintedHandoff 0 0 46 0
>  0
> MiscStage 0 0  0 0
>  0
> CompactionExecutor866  68124 0
>  0
> MemtableReclaimMemory 0 0166 0
>  0
> PendingRangeCalculator0 0 38 0
>  0
> GossipStage   0 0 242455 0
>  0
> MigrationStage0 0  0 0
>  0
> MemtablePostFlush 0 0   3682 0
>  0
> ValidationExecutor0 0   2246 0
>  0
> Sampler   0 0  0 0
>  0
> MemtableFlushWriter   0 0166 0
>  0
> InternalResponseStage 0 0   8866 0
>  0
> AntiEntropyStage  0 0  15417 0
>  0
> Repair#7  0 0160 0
>  0
> CacheCleanupExecutor  0 0  0 0
>  0
> Native-Transport-Requests 0 0 327334 0
>  0
> Message type   Dropped
> READ 0
> RANGE_SLICE  0
> _TRACE   0
> MUTATION 0
> COUNTER_MUTATION 0
> REQUEST_RESPONSE 0
> PAGED_RANGE  0
> READ_REPAIR  0
> {code}
> {code} nodetool netstats{code} shows some pending messages which never get 
> processed and noting in progress -
> {code}
> Mode: NORMAL
> Not sending any streams.
> Read Repair Statistics:
> Attempted: 15585
> Mismatch 

[jira] [Commented] (CASSANDRA-12659) Query in reversed order brough back deleted data

2016-09-17 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12659?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15499215#comment-15499215
 ] 

Wei Deng commented on CASSANDRA-12659:
--

Can you reliably reproduce this problem? If yes, it would be very helpful if 
you describe the repro steps so that somebody can take a closer look.

BTW, "there are no tombstones left" doesn't mean you're safe. You may have run 
into a typical ["zombie 
resurrection"|http://thelastpickle.com/blog/2016/07/27/about-deletes-and-tombstones.html]
 situation when the system cleared up all of the tombstones.

> Query in reversed order brough back deleted data
> 
>
> Key: CASSANDRA-12659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12659
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 3.0.5, 6 nodes cluster
>Reporter: Tai Khuu Tan
>
> We have and issues with our Cassandra 3.0.5. After we deleted a large amount 
> of data in the multiple partition keys. Query those partition keys with 
> reversed order on a clustering key return the deleted data. I have checked 
> and there are no tombstones left. All of them are deleted. So I don't know 
> where or how can those deleted data still exist. Is there any other place 
> that Cassandra will read data when query in reverse order compare to normal 
> order ?
> the schema is very simple
> {noformat}
> CREATE TABLE table ( uid varchar, version timestamp, data1 varchar, data2 
> varchar, data3 varchar, data4 varchar, data5 varchar, PRIMARY KEY (uid, 
> version, data1 , data2 , data3 , data4 ) ) with compact storage;
> {noformat}
> Query are doing reverse order on column timestamp
> Ex:
> {noformat}
> select * from data where uid="uid1" order by version DESC
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12615) Improve LCS compaction concurrency during L0->L1 compaction

2016-09-05 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12615:
-
Labels: lcs  (was: )

> Improve LCS compaction concurrency during L0->L1 compaction
> ---
>
> Key: CASSANDRA-12615
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12615
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs
> Attachments: L0_L1_inefficiency.jpg
>
>
> I've done multiple experiments with {{compaction-stress}} at 100GB, 200GB, 
> 400GB and 600GB levels. These scenarios share a common pattern: at the 
> beginning of the compaction they all overwhelm L0 with a lot (hundreds to 
> thousands) of 128MB SSTables. One common observation I noticed from 
> visualizing the compaction.log files from these tests is that initially after 
> some massive STCS-in-L0 activities (could take up to 40% of total compaction 
> time), L0->L1 always takes a really long time which frequently involves all 
> of the bigger L0 SSTables (the results of many STCS compactions earlier) and 
> all of the 10 L1 SSTables, and the output covers almost the full data set. 
> Since L0->L1 can only happen single-threaded, we often spend close to 40% of 
> the total compaction time in this L0->L1 stage, and only after this first 
> really long L0->L1 finishes and 100s or 1000s of SSTables land on L1, can 
> concurrent compactions at higher levels resume (to move the thousands of L1 
> SSTables to higher levels). The attached snapshot demonstrates this 
> observation.
> The question is, if this L0->L1 compaction is so big and can only happen 
> single-threaded, and ends up generating thousands of L1 SSTables, most of 
> which will have to up-level later anyway (as L1 can accommodate at most 10 
> SSTables), why not start that L1+ up-level earlier, i.e. before this L0->L1 
> compaction finishes.
> I can think of a few approaches: 1) break L0->L1 into smaller chunks if it 
> realizes that the output of such L0->L1 compaction is going to far exceed the 
> capacity of L1, this will allow each L0->L1 to finish sooner, and have the 
> resulting L1 SSTables to be able to participate in higher up-level 
> activities; 2) still treating the full L0->L1 as one big compaction session, 
> but making the intermediate results (once the number of L1 SSTable output 
> exceeds the L1 capacity) available for higher up-level activities.
> If we can somehow leverage more threads during this massive L0->L1 phase, we 
> can save close to 40% of the total compaction time when L0 is initially 
> backlogged, which will be a great improvement to our LCS compaction 
> throughput.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12615) Improve LCS compaction concurrency during L0->L1 compaction

2016-09-05 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12615:


 Summary: Improve LCS compaction concurrency during L0->L1 
compaction
 Key: CASSANDRA-12615
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12615
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Wei Deng
 Attachments: L0_L1_inefficiency.jpg

I've done multiple experiments with {{compaction-stress}} at 100GB, 200GB, 
400GB and 600GB levels. These scenarios share a common pattern: at the 
beginning of the compaction they all overwhelm L0 with a lot (hundreds to 
thousands) of 128MB SSTables. One common observation I noticed from visualizing 
the compaction.log files from these tests is that initially after some massive 
STCS-in-L0 activities (could take up to 40% of total compaction time), L0->L1 
always takes a really long time which frequently involves all of the bigger L0 
SSTables (the results of many STCS compactions earlier) and all of the 10 L1 
SSTables, and the output covers almost the full data set. Since L0->L1 can only 
happen single-threaded, we often spend close to 40% of the total compaction 
time in this L0->L1 stage, and only after this first really long L0->L1 
finishes and 100s or 1000s of SSTables land on L1, can concurrent compactions 
at higher levels resume (to move the thousands of L1 SSTables to higher 
levels). The attached snapshot demonstrates this observation.

The question is, if this L0->L1 compaction is so big and can only happen 
single-threaded, and ends up generating thousands of L1 SSTables, most of which 
will have to up-level later anyway (as L1 can accommodate at most 10 SSTables), 
why not start that L1+ up-level earlier, i.e. before this L0->L1 compaction 
finishes.

I can think of a few approaches: 1) break L0->L1 into smaller chunks if it 
realizes that the output of such L0->L1 compaction is going to far exceed the 
capacity of L1, this will allow each L0->L1 to finish sooner, and have the 
resulting L1 SSTables to be able to participate in higher up-level activities; 
2) still treating the full L0->L1 as one big compaction session, but making the 
intermediate results (once the number of L1 SSTable output exceeds the L1 
capacity) available for higher up-level activities.

If we can somehow leverage more threads during this massive L0->L1 phase, we 
can save close to 40% of the total compaction time when L0 is initially 
backlogged, which will be a great improvement to our LCS compaction throughput.





--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12613) Expose the inter_dc_stream_throughput_outbound_megabits_per_sec in "nodetool setstreamthroughput" interface

2016-09-05 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12613:


 Summary: Expose the 
inter_dc_stream_throughput_outbound_megabits_per_sec in "nodetool 
setstreamthroughput" interface
 Key: CASSANDRA-12613
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12613
 Project: Cassandra
  Issue Type: Improvement
  Components: Configuration
Reporter: Wei Deng
Priority: Minor


We added inter_dc_stream_throughput_outbound_megabits_per_sec in CASSANDRA-6596 
to differentiate intra-DC and inter-DC streaming throughput restrictions, 
however, it's never exposed via the nodetool interface. This JIRA is a request 
to allow this new parameter to be set from nodetool (in addition to the 
cassandra.yaml and JMX method).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-6698) Many too small SSTables when full repair

2016-09-05 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-6698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-6698:

Labels: lcs  (was: )

> Many too small SSTables when full repair
> 
>
> Key: CASSANDRA-6698
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6698
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Roman Skvazh
>Priority: Minor
>  Labels: lcs
> Attachments: cassa-many-small-sstables.txt
>
>
> We have troubles when cassandra drops messages because there is too many 
> (over 10,000 on one column family) small (from 1Kb to 200Kb, and normal sizes 
> too) and many pending compactions (over 700).
> We are using Leveled compaction with 160Mb sstable size.
> PS. Temp fix: stop repair, disable thrift,gossip and wait for compactions to 
> be finished. Because this, we can not run full repair for about a month :(



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-9033) Upgrading from 2.1.1 to 2.1.3 with LCS and many sstable files makes nodes unresponsive

2016-09-05 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-9033?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-9033:

Labels:   (was: lcs)

> Upgrading from 2.1.1 to 2.1.3 with LCS  and many sstable files makes nodes 
> unresponsive
> ---
>
> Key: CASSANDRA-9033
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9033
> Project: Cassandra
>  Issue Type: Bug
> Environment: * Ubuntu 14.04.2 - Linux ip-10-0-2-122 3.13.0-46-generic 
> #79-Ubuntu SMP Tue Mar 10 20:06:50 UTC 2015 x86_64 x86_64 x86_64 GNU/Linux
> * EC2 m2-xlarge instances [4cpu, 16GB RAM, 1TB storage on 3 platters]
> * 12 nodes running a mix of 2.1.1 and 2.1.3
> * 8GB stack size with offheap objects
>Reporter: Brent Haines
>Assignee: Marcus Eriksson
> Attachments: cassandra-env.sh, cassandra.yaml, system.log.1.zip
>
>
> We have an Event Log table using LCS that has grown fast. There are more than 
> 100K sstable files that are around 1KB. Increasing compactors and adjusting 
> compaction throttling upward doesn't make a difference. It has been running 
> great though until we upgraded to 2.1.3. Those nodes needed more RAM for the 
> stack (12 GB) to even have a prayer of responding to queries. They bog down 
> and become unresponsive. There are no GC messages that I can see, and no 
> compaction either. 
> The only work-around I have found is to decommission, blow away the big CF 
> and rejoin. That happens in about 20 minutes and everything is freaking happy 
> again. The size of the files is more like what I'd expect as well. 
> Our schema: 
> {code}
> cqlsh> describe columnfamily data.stories
> CREATE TABLE data.stories (
> id timeuuid PRIMARY KEY,
> action_data timeuuid,
> action_name text,
> app_id timeuuid,
> app_instance_id timeuuid,
> data map,
> objects set,
> time_stamp timestamp,
> user_id timeuuid
> ) WITH bloom_filter_fp_chance = 0.01
> AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}'
> AND comment = 'Stories represent the timeline and are placed in the 
> dashboard for the brand manager to see'
> AND compaction = {'min_threshold': '4', 'class': 
> 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 
> 'max_threshold': '32'}
> AND compression = {'sstable_compression': 
> 'org.apache.cassandra.io.compress.LZ4Compressor'}
> AND dclocal_read_repair_chance = 0.1
> AND default_time_to_live = 0
> AND gc_grace_seconds = 864000
> AND max_index_interval = 2048
> AND memtable_flush_period_in_ms = 0
> AND min_index_interval = 128
> AND read_repair_chance = 0.0
> AND speculative_retry = '99.0PERCENTILE';
> cqlsh> 
> {code}
> There were no log entries that stood out. It pretty much consisted of "x is 
> down" "x is up" repeated ad infinitum. I have attached the zipped system.log 
> that has the situation after the upgrade and then after I stopped, removed 
> system, system_traces, OpsCenter, and data/stories-/* and restarted. 
> It has rejoined the cluster now and is busy read-repairing to recover its 
> data.
> On another note, we see a lot of this during repair now (on all the nodes): 
> {code}
> ERROR [AntiEntropySessions:5] 2015-03-24 20:03:10,207 RepairSession.java:303 
> - [repair #c5043c40-d260-11e4-a2f2-8bb3e2bbdb35] session completed with the 
> following error
> java.io.IOException: Failed during snapshot creation.
> at 
> org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
> at 
> org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:146) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
> at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
> ~[guava-16.0.jar:na]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  [na:1.7.0_55]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_55]
> at java.lang.Thread.run(Thread.java:745) [na:1.7.0_55]
> ERROR [AntiEntropySessions:5] 2015-03-24 20:03:10,208 
> CassandraDaemon.java:167 - Exception in thread 
> Thread[AntiEntropySessions:5,5,RMI Runtime]
> java.lang.RuntimeException: java.io.IOException: Failed during snapshot 
> creation.
> at com.google.common.base.Throwables.propagate(Throwables.java:160) 
> ~[guava-16.0.jar:na]
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> ~[na:1.7.0_55]
> at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> ~[na:1.7.0_55]
> at 
> 

[jira] [Created] (CASSANDRA-12610) Introduce a mixed write + compact mode for compaction-stress to simulate more realistic workload

2016-09-04 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12610:


 Summary: Introduce a mixed write + compact mode for 
compaction-stress to simulate more realistic workload
 Key: CASSANDRA-12610
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12610
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Wei Deng


The new offline stress utility {{compaction-stress}} was introduced in 
CASSANDRA-11844 which greatly simplified the amount of work needed to perform a 
compaction experiment. However, it currently only provides two modes {{write}} 
and {{compact}}, which means the user will always have to use {{write}} to 
generate a bunch of SSTables first, and then use {{compact}} to compact them 
using desired compaction strategy. This is not as close to the real workload as 
possible where new SSTable writing and compaction can often happen 
simultaneously which puts a different strain on CPU and IO than the current 
test pattern.

With the introduction of the mixed (write / compact) mode proposed in this 
JIRA, we should at least allow parameters to specify how big each newly-written 
SSTable is (already supported by {{buffer-size-mb}}), and how frequently the 
write happens (this is the most straightforward way to simulate a memtable 
flush and might over-simplify the flush behavior, but should be easier to 
implement; the more sophisticated SSTable write behavior probably can be done 
in a follow-up JIRA), and allow specifying how many threads allocated to 
{{write}} and {{compact}} respectively.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12526) For LCS, single SSTable up-level is handled inefficiently

2016-09-04 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12526:
-
Issue Type: Improvement  (was: Bug)

> For LCS, single SSTable up-level is handled inefficiently
> -
>
> Key: CASSANDRA-12526
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12526
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: compaction, lcs, performance
>
> I'm using the latest trunk (as of August 2016, which probably is going to be 
> 3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
> inefficiency.
> The test data is generated using cassandra-stress default parameters 
> (keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
> inserted partitions that will never merge in compactions, which is probably 
> the worst kind of workload for LCS (however, I'll detail later why this 
> scenario should not be ignored as a corner case; for now, let's just assume 
> we still want to handle this scenario efficiently).
> After the compaction test is done, I scrubbed debug.log for patterns that 
> match  the "Compacted" summary so that I can see how long each individual 
> compaction took and how many bytes they processed. The search pattern is like 
> the following:
> {noformat}
> grep 'Compacted.*standard1' debug.log
> {noformat}
> Interestingly, I noticed a lot of the finished compactions are marked as 
> having *only one* SSTable involved. With the workload mentioned above, the 
> "single SSTable" compactions actually consist of the majority of all 
> compactions (as shown below), so its efficiency can affect the overall 
> compaction throughput quite a bit.
> {noformat}
> automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
> debug.log-test1 | wc -l
> 243
> automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
> debug.log-test1 | grep ") 1 sstable" | wc -l
> 218
> {noformat}
> By looking at the code, it appears that there's a way to directly edit the 
> level of a particular SSTable like the following:
> {code}
> sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 
> targetLevel);
> sstable.reloadSSTableMetadata();
> {code}
> To be exact, I summed up the time spent for these single-SSTable compactions 
> (the total data size is 60GB) and found that if each compaction only needs to 
> spend 100ms for only the metadata change (instead of the 10+ second they're 
> doing now), it can already achieve 22.75% saving on total compaction time.
> Compared to what we have now (reading the whole single-SSTable from old level 
> and writing out the same single-SSTable at the new level), the only 
> difference I could think of by using this approach is that the new SSTable 
> will have the same file name (sequence number) as the old one's, which could 
> break some assumptions on some other part of the code. However, not having to 
> go through the full read/write IO, and not having to bear the overhead of 
> cleaning up the old file, creating the new file, creating more churns in heap 
> and file buffer, it seems the benefits outweigh the inconvenience. So I'd 
> argue this JIRA belongs to LHF and should be made available in 3.0.x as well.
> As mentioned in the 2nd paragraph, I'm also going to address why this kind of 
> all-new-partition workload should not be ignored as a corner case. Basically, 
> for the main use case of LCS where you need to frequently merge partitions to 
> optimize read and eliminate tombstones and expired data sooner, LCS can be 
> perfectly happy and efficiently perform the partition merge and tombstone 
> elimination for a long time. However, as soon as the node becomes a bit 
> unhealthy for various reasons (could be a bad disk so it's missing a whole 
> bunch of mutations and need repair, could be the user chooses to ingest way 
> more data than it usually takes and exceeds its capability, or god-forbidden, 
> some DBA chooses to run offline sstablelevelreset), you will have to handle 
> this kind of "all-new-partition with a lot of SSTables in L0" scenario, and 
> once all L0 SSTables finally gets up-leveled to L1, you will likely see a lot 
> of such single-SSTable compactions, which is the situation this JIRA is 
> intended to address.
> Actually, when I think more about this, to make this kind of single SSTable 
> up-level more efficient will not only help the all-new-partition scenario, 
> but also help in general any time when there is a big backlog of L0 SSTables 
> due to too many flushes or excessive repair streaming with vnode. In those 
> situations, by default STCS_in_L0 will be triggered, and you will end up 
> getting a bunch of much bigger L0 SSTables after STCS is 

[jira] [Updated] (CASSANDRA-8190) Compactions stop completely because of RuntimeException in CompactionExecutor

2016-09-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8190:

Labels: lcs  (was: )

> Compactions stop completely because of RuntimeException in CompactionExecutor
> -
>
> Key: CASSANDRA-8190
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8190
> Project: Cassandra
>  Issue Type: Bug
> Environment: DSE 4.5.2 (Cassandra 2.0.10)
>Reporter: Nikolai Grigoriev
>Assignee: Marcus Eriksson
>  Labels: lcs
> Attachments: cassandra-env.sh, cassandra.yaml, jstack.txt.gz, 
> system.log.gz, system.log.gz
>
>
> I have a cluster that is recovering from being overloaded with writes.  I am 
> using the workaround from CASSANDRA-6621 to prevent the STCS fallback (which 
> is killing the cluster - see CASSANDRA-7949). 
> I have observed that after one or more exceptions like this
> {code}
> ERROR [CompactionExecutor:4087] 2014-10-26 22:50:05,016 CassandraDaemon.java 
> (line 199) Exception in thread Thread[CompactionExecutor:4087,1,main]
> java.lang.RuntimeException: Last written key DecoratedKey(425124616570337476, 
> 0010033523da10033523da10
> 400100) >= current key DecoratedKey(-8778432288598355336, 
> 0010040c7a8f10040c7a8f10
> 400100) writing into 
> /cassandra-data/disk2/myks/mytable/myks-mytable-tmp-jb-130379-Data.db
> at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
> at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
> at 
> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
> at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> the node completely stops the compactions and I end up in the state like this:
> {code}
> # nodetool compactionstats
> pending tasks: 1288
>   compaction typekeyspace   table   completed 
>   total  unit  progress
> Active compaction remaining time :n/a
> {code}
> The node recovers if restarted and starts compactions - until getting more 
> exceptions like this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12608) For LCS compaction sessions, have compactionstats to print out src_level and dst_level

2016-09-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12608:
-
Description: It seems that we use reportCompactionTable() function in 
o.a.c.tools.nodetool.CompactionStats to print out the information for each 
compaction session, which already has access to compactionID according to [this 
line|https://github.com/apache/cassandra/blob/9895837557137b96bb12d371e42138770539e82d/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java#L92].
 This means it should not be hard to get the source level and destination level 
from a LCS compaction ID. If we could add two additional columns "src_level" 
and "dst_level" whenever we detect a particular compaction session is using 
LeveledCompactionStrategy, it would be very helpful in understanding the 
progress LCS is making and what all of the concurrent compaction sessions are 
doing for people observing the compactions in real-time. For none-LCS 
compaction sessions, these two columns can just print empty strings.  (was: It 
seems that we use reportCompactionTable() function in 
o.a.c.tools.nodetool.CompactionStats to print out the information for each 
compaction session, which already has access to compactionID according to [this 
line|https://github.com/apache/cassandra/blob/9895837557137b96bb12d371e42138770539e82d/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java#L92].
 This means it should not be hard to get the source level and destination level 
from a LCS compaction ID. If we could add two additional columns "src_level" 
and "dst_level" whenever we detect a particular compaction session is using 
LeveledCompactionStrategy, it would be very helpful in understanding the 
progress LCS is making and what all of the concurrent compaction sessions are 
doing. For none-LCS compaction sessions, these two columns can just print empty 
strings.)

> For LCS compaction sessions, have compactionstats to print out src_level and 
> dst_level
> --
>
> Key: CASSANDRA-12608
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12608
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>Priority: Minor
>  Labels: lcs, observation
>
> It seems that we use reportCompactionTable() function in 
> o.a.c.tools.nodetool.CompactionStats to print out the information for each 
> compaction session, which already has access to compactionID according to 
> [this 
> line|https://github.com/apache/cassandra/blob/9895837557137b96bb12d371e42138770539e82d/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java#L92].
>  This means it should not be hard to get the source level and destination 
> level from a LCS compaction ID. If we could add two additional columns 
> "src_level" and "dst_level" whenever we detect a particular compaction 
> session is using LeveledCompactionStrategy, it would be very helpful in 
> understanding the progress LCS is making and what all of the concurrent 
> compaction sessions are doing for people observing the compactions in 
> real-time. For none-LCS compaction sessions, these two columns can just print 
> empty strings.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12608) For LCS compaction sessions, have compactionstats to print out src_level and dst_level

2016-09-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12608:
-
Labels: lcs observation  (was: )

> For LCS compaction sessions, have compactionstats to print out src_level and 
> dst_level
> --
>
> Key: CASSANDRA-12608
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12608
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>Priority: Minor
>  Labels: lcs, observation
>
> It seems that we use reportCompactionTable() function in 
> o.a.c.tools.nodetool.CompactionStats to print out the information for each 
> compaction session, which already has access to compactionID according to 
> [this 
> line|https://github.com/apache/cassandra/blob/9895837557137b96bb12d371e42138770539e82d/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java#L92].
>  This means it should not be hard to get the source level and destination 
> level from a LCS compaction ID. If we could add two additional columns 
> "src_level" and "dst_level" whenever we detect a particular compaction 
> session is using LeveledCompactionStrategy, it would be very helpful in 
> understanding the progress LCS is making and what all of the concurrent 
> compaction sessions are doing. For none-LCS compaction sessions, these two 
> columns can just print empty strings.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12608) For LCS compaction sessions, have compactionstats to print out src_level and dst_level

2016-09-03 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12608:


 Summary: For LCS compaction sessions, have compactionstats to 
print out src_level and dst_level
 Key: CASSANDRA-12608
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12608
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Wei Deng
Priority: Minor


It seems that we use reportCompactionTable() function in 
o.a.c.tools.nodetool.CompactionStats to print out the information for each 
compaction session, which already has access to compactionID according to [this 
line|https://github.com/apache/cassandra/blob/9895837557137b96bb12d371e42138770539e82d/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java#L92].
 This means it should not be hard to get the source level and destination level 
from a LCS compaction ID. If we could add two additional columns "src_level" 
and "dst_level" whenever we detect a particular compaction session is using 
LeveledCompactionStrategy, it would be very helpful in understanding the 
progress LCS is making and what all of the concurrent compaction sessions are 
doing. For none-LCS compaction sessions, these two columns can just print empty 
strings.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-02 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15456278#comment-15456278
 ] 

Wei Deng edited comment on CASSANDRA-12591 at 9/2/16 10:33 PM:
---

So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

The above tests were conducted using the default parameters from 
compaction-stress which generates unique partitions for all writes, so it 
reflects the worst kind of workload for LCS. Considering this, I also conducted 
another set of tests using {{"partition-count=1000"}} to force 
compaction-stress to generate a lot of overwrites for the same partitions. 
While keeping everything else to same and adding this 
{{"partition-count=1000"}} parameter, the overall compaction times with 1280MB 
max_sstable_size are: 4m59.307s, 4m52.002s, 5m0.967s; the overall compaction 
times with 160MB max_sstable_size are: 6m11.533s, 6m21.200s, 6m10.904s. These 
numbers are understandably faster than the "all unique partition" scenario in 
the last paragraph, and if you calculate the average seconds, 1280MB 
max_sstable_size is 21% faster than 160MB max_sstable_size.

I realize 10GB data is barely enough to test 1280MB sstable size as the data 
will only go from L0->L1, so the next run I'm going to use 100GB data size on 
this hardware (keeping everything else the same) and see how the numbers 
compare.


was (Author: weideng):
So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

The above tests were conducted using the default parameters from 
compaction-stress which generates unique partitions for all writes, so it 
reflects the worst kind of workload for LCS. Considering this, I also conducted 
another set of tests using {{"--partition-count=1000"}} to force 
compaction-stress to generate a lot of overwrites for the same 

[jira] [Comment Edited] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-02 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15456278#comment-15456278
 ] 

Wei Deng edited comment on CASSANDRA-12591 at 9/2/16 10:32 PM:
---

So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

The above tests were conducted using the default parameters from 
compaction-stress which generates unique partitions for all writes, so it 
reflects the worst kind of workload for LCS. Considering this, I also conducted 
another set of tests using "--partition-count=1000" to force compaction-stress 
to generate a lot of overwrites for the same partitions. While keeping 
everything else to same and adding this "--partition-count=1000" parameter, the 
overall compaction times with 1280MB max_sstable_size are: 4m59.307s, 
4m52.002s, 5m0.967s; the overall compaction times with 160MB max_sstable_size 
are: 6m11.533s, 6m21.200s, 6m10.904s. These numbers are understandably faster 
than the "all unique partition" scenario in the last paragraph, and if you 
calculate the average seconds, 1280MB max_sstable_size is 21% faster than 160MB 
max_sstable_size.

I realize 10GB data is barely enough to test 1280MB sstable size as the data 
will only go from L0->L1, so the next run I'm going to use 100GB data size on 
this hardware (keeping everything else the same) and see how the numbers 
compare.


was (Author: weideng):
So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

I realize 10GB data is barely enough to test 1280MB sstable size as the data 
will only go from L0->L1, so the next run I'm going to use 100GB data size on 
this hardware (keeping everything else the same) and see how the numbers 
compare.

> Re-evaluate the default 160MB sstable_size_in_mb choice in LCS
> 

[jira] [Comment Edited] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-02 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15456278#comment-15456278
 ] 

Wei Deng edited comment on CASSANDRA-12591 at 9/2/16 10:32 PM:
---

So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

The above tests were conducted using the default parameters from 
compaction-stress which generates unique partitions for all writes, so it 
reflects the worst kind of workload for LCS. Considering this, I also conducted 
another set of tests using {{"--partition-count=1000"}} to force 
compaction-stress to generate a lot of overwrites for the same partitions. 
While keeping everything else to same and adding this 
{{"--partition-count=1000"}} parameter, the overall compaction times with 
1280MB max_sstable_size are: 4m59.307s, 4m52.002s, 5m0.967s; the overall 
compaction times with 160MB max_sstable_size are: 6m11.533s, 6m21.200s, 
6m10.904s. These numbers are understandably faster than the "all unique 
partition" scenario in the last paragraph, and if you calculate the average 
seconds, 1280MB max_sstable_size is 21% faster than 160MB max_sstable_size.

I realize 10GB data is barely enough to test 1280MB sstable size as the data 
will only go from L0->L1, so the next run I'm going to use 100GB data size on 
this hardware (keeping everything else the same) and see how the numbers 
compare.


was (Author: weideng):
So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

The above tests were conducted using the default parameters from 
compaction-stress which generates unique partitions for all writes, so it 
reflects the worst kind of workload for LCS. Considering this, I also conducted 
another set of tests using "--partition-count=1000" to force compaction-stress 
to generate a lot of overwrites for the same 

[jira] [Comment Edited] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-01 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15457121#comment-15457121
 ] 

Wei Deng edited comment on CASSANDRA-12591 at 9/2/16 12:56 AM:
---

Just finished a 100GB compaction test on the same hardware, and it still shows 
that 1280MB sstable size indeed works better than 160MB.

I only had time to finish one run and here are the numbers:

1280MB sstable size: 127m2.955s
160MB sstable size: 162m46.877s

So 1280MB max_sstable_size is again 22% improvement on compaction throughput.

Next I'm going to run the same 100GB tests on a SSD-based environment (Amazon 
i2.xlarge) to see if the same advantage still remains.


was (Author: weideng):
Just finished a 100GB compaction test on the same hardware, and it still shows 
that 1280MB sstable size works much better than 160MB.

I only had time to finish one run and here are the numbers:

1280MB sstable size: 127m2.955s
160MB sstable size: 162m46.877s

So 1280MB max_sstable_size is again 22% improvement on compaction throughput.

Next I'm going to run the same 100GB tests on a SSD-based environment (Amazon 
i2.xlarge) to see if the same advantage still remains.

> Re-evaluate the default 160MB sstable_size_in_mb choice in LCS
> --
>
> Key: CASSANDRA-12591
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12591
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs
>
> There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
> the best max_sstable_size used by LeveledCompactionStrategy, and the 
> conclusion derived from that effort was to use 160MB as the most optimal size 
> for both throughput (i.e. the time spent on compaction, the smaller the 
> better) and the amount of bytes compacted (to avoid write amplification, the 
> less the better).
> However, when I read more into that test report (the short 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-5727?focusedCommentId=13722571=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13722571]
>  describing the tests), I realized it was conducted on a hardware with the 
> following configuration: "a single rackspace node with 2GB of ram." I'm not 
> sure if this was an ok hardware configuration for production Cassandra 
> deployment at that time (mid-2013), but it is definitely far lower from 
> today's hardware standard now.
> Given that we now have compaction-stress which is able to generate SSTables 
> based on user defined stress profile with user defined table schema and 
> compaction parameters (compatible to cassandra-stress), it would be a useful 
> effort to relook at this number using a more realistic hardware configuration 
> and see if 160MB is still the optimal choice. It might also impact our 
> perceived "practical" node density with LCS nodes if it turns out bigger 
> max_sstable_size actually works better as it will allow less number of 
> SSTables (and hence less level and less write amplification) per node with 
> bigger density.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-01 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15457121#comment-15457121
 ] 

Wei Deng commented on CASSANDRA-12591:
--

Just finished a 100GB compaction test on the same hardware, and it still shows 
that 1280MB sstable size works much better than 160MB.

I only had time to finish one run and here are the numbers:

1280MB sstable size: 127m2.955s
160MB sstable size: 162m46.877s

So 1280MB max_sstable_size is again 22% improvement on compaction throughput.

Next I'm going to run the same 100GB tests on a SSD-based environment (Amazon 
i2.xlarge) to see if the same advantage still remains.

> Re-evaluate the default 160MB sstable_size_in_mb choice in LCS
> --
>
> Key: CASSANDRA-12591
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12591
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs
>
> There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
> the best max_sstable_size used by LeveledCompactionStrategy, and the 
> conclusion derived from that effort was to use 160MB as the most optimal size 
> for both throughput (i.e. the time spent on compaction, the smaller the 
> better) and the amount of bytes compacted (to avoid write amplification, the 
> less the better).
> However, when I read more into that test report (the short 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-5727?focusedCommentId=13722571=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13722571]
>  describing the tests), I realized it was conducted on a hardware with the 
> following configuration: "a single rackspace node with 2GB of ram." I'm not 
> sure if this was an ok hardware configuration for production Cassandra 
> deployment at that time (mid-2013), but it is definitely far lower from 
> today's hardware standard now.
> Given that we now have compaction-stress which is able to generate SSTables 
> based on user defined stress profile with user defined table schema and 
> compaction parameters (compatible to cassandra-stress), it would be a useful 
> effort to relook at this number using a more realistic hardware configuration 
> and see if 160MB is still the optimal choice. It might also impact our 
> perceived "practical" node density with LCS nodes if it turns out bigger 
> max_sstable_size actually works better as it will allow less number of 
> SSTables (and hence less level and less write amplification) per node with 
> bigger density.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12596) cosmetic issue with compaction-stress write

2016-09-01 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12596:


 Summary: cosmetic issue with compaction-stress write
 Key: CASSANDRA-12596
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12596
 Project: Cassandra
  Issue Type: Bug
Reporter: Wei Deng
Priority: Minor


I got the following when I was trying to run `compaction-stress write` with a 
10GB data set. Looks like a race condition with the lifecycle transaction log 
but it only affects the reporting of the progress temporarily.

{noformat}
Written 50.27GB of 100GB
ERROR 23:49:55,843 Error while getting 
/mnt/compaction/stresscql/blogposts-606839b5bd683964ad7f57f519eb6dae folder 
size. {}
java.nio.file.NoSuchFileException: 
/mnt/compaction/stresscql/blogposts-606839b5bd683964ad7f57f519eb6dae/mc_txn_write_c6877a90-709e-11e6-91a5-a1533b002bc1.log
at 
sun.nio.fs.UnixException.translateToIOException(UnixException.java:86) 
~[na:1.8.0_66]
at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) 
~[na:1.8.0_66]
at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) 
~[na:1.8.0_66]
at 
sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55)
 ~[na:1.8.0_66]
at 
sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:144)
 ~[na:1.8.0_66]
at 
sun.nio.fs.LinuxFileSystemProvider.readAttributes(LinuxFileSystemProvider.java:99)
 ~[na:1.8.0_66]
at java.nio.file.Files.readAttributes(Files.java:1737) ~[na:1.8.0_66]
at java.nio.file.FileTreeWalker.getAttributes(FileTreeWalker.java:219) 
~[na:1.8.0_66]
at java.nio.file.FileTreeWalker.visit(FileTreeWalker.java:276) 
~[na:1.8.0_66]
at java.nio.file.FileTreeWalker.next(FileTreeWalker.java:372) 
~[na:1.8.0_66]
at java.nio.file.Files.walkFileTree(Files.java:2706) ~[na:1.8.0_66]
at java.nio.file.Files.walkFileTree(Files.java:2742) ~[na:1.8.0_66]
at 
org.apache.cassandra.io.util.FileUtils.folderSize(FileUtils.java:478) 
~[main/:na]
at 
org.apache.cassandra.db.Directories.getRawDiretoriesSize(Directories.java:939) 
[main/:na]
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:322)
 [stress/:na]
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353) 
[stress/:na]
Written 37.49GB of 100GB
{noformat}

You will see the next "Written xxxGB" number being inaccurate because it 
interrupted the directory accounting of the data files, but the subsequent 
"Written xxxGB" number will get back to normal.

Since it generates a ERROR log entry it's probably still worthwhile to fix.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12526) For LCS, single SSTable up-level is handled inefficiently

2016-09-01 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12526:
-
Description: 
I'm using the latest trunk (as of August 2016, which probably is going to be 
3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
inefficiency.

The test data is generated using cassandra-stress default parameters 
(keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
inserted partitions that will never merge in compactions, which is probably the 
worst kind of workload for LCS (however, I'll detail later why this scenario 
should not be ignored as a corner case; for now, let's just assume we still 
want to handle this scenario efficiently).

After the compaction test is done, I scrubbed debug.log for patterns that match 
 the "Compacted" summary so that I can see how long each individual compaction 
took and how many bytes they processed. The search pattern is like the 
following:

{noformat}
grep 'Compacted.*standard1' debug.log
{noformat}

Interestingly, I noticed a lot of the finished compactions are marked as having 
*only one* SSTable involved. With the workload mentioned above, the "single 
SSTable" compactions actually consist of the majority of all compactions (as 
shown below), so its efficiency can affect the overall compaction throughput 
quite a bit.

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | wc -l
243
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | grep ") 1 sstable" | wc -l
218
{noformat}

By looking at the code, it appears that there's a way to directly edit the 
level of a particular SSTable like the following:

{code}
sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 
targetLevel);
sstable.reloadSSTableMetadata();
{code}

To be exact, I summed up the time spent for these single-SSTable compactions 
(the total data size is 60GB) and found that if each compaction only needs to 
spend 100ms for only the metadata change (instead of the 10+ second they're 
doing now), it can already achieve 22.75% saving on total compaction time.

Compared to what we have now (reading the whole single-SSTable from old level 
and writing out the same single-SSTable at the new level), the only difference 
I could think of by using this approach is that the new SSTable will have the 
same file name (sequence number) as the old one's, which could break some 
assumptions on some other part of the code. However, not having to go through 
the full read/write IO, and not having to bear the overhead of cleaning up the 
old file, creating the new file, creating more churns in heap and file buffer, 
it seems the benefits outweigh the inconvenience. So I'd argue this JIRA 
belongs to LHF and should be made available in 3.0.x as well.

As mentioned in the 2nd paragraph, I'm also going to address why this kind of 
all-new-partition workload should not be ignored as a corner case. Basically, 
for the main use case of LCS where you need to frequently merge partitions to 
optimize read and eliminate tombstones and expired data sooner, LCS can be 
perfectly happy and efficiently perform the partition merge and tombstone 
elimination for a long time. However, as soon as the node becomes a bit 
unhealthy for various reasons (could be a bad disk so it's missing a whole 
bunch of mutations and need repair, could be the user chooses to ingest way 
more data than it usually takes and exceeds its capability, or god-forbidden, 
some DBA chooses to run offline sstablelevelreset), you will have to handle 
this kind of "all-new-partition with a lot of SSTables in L0" scenario, and 
once all L0 SSTables finally gets up-leveled to L1, you will likely see a lot 
of such single-SSTable compactions, which is the situation this JIRA is 
intended to address.

Actually, when I think more about this, to make this kind of single SSTable 
up-level more efficient will not only help the all-new-partition scenario, but 
also help in general any time when there is a big backlog of L0 SSTables due to 
too many flushes or excessive repair streaming with vnode. In those situations, 
by default STCS_in_L0 will be triggered, and you will end up getting a bunch of 
much bigger L0 SSTables after STCS is done. When it's time to up-level those 
much bigger L0 SSTables most likely they will overlap among themselves and you 
will add them all into your compaction session (along with all overlapped L1 
SSTables). For these much bigger L0 SSTables, they have gone through a few 
rounds of STCS compactions, so if there's partition merge that needs to be done 
because fragments of the same partition are dispersed in smaller L0 SSTables 
earlier, after those STCS rounds, what you end up having in those much bigger 
L0 SSTables (generated by STCS) will not have much more 

[jira] [Commented] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-01 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15456278#comment-15456278
 ] 

Wei Deng commented on CASSANDRA-12591:
--

So I've done some quick initial tests using latest trunk (i.e. C* 3.10) code 
just to prove the point whether this is a worthwhile effort. The hardware I'm 
using is still not a typical/adequate-enough configuration I'd use for a 
production Cassandra deployment (GCE n1-standard-4, with 4 vCPUs, 15GB RAM and 
a single 1TB persistent disk that's spindle-based), but I'm already seeing a 
positive sign that shows bigger max_sstable_size can be helpful for compaction 
throughput.

Based on the initial results (at each max_sstable_size, I did three runs from 
scratch; for all runs I set compaction threads to 4, and since there will be no 
throttling enforced by compaction-stress the setting would be equivalent to 
setting compaction_throughput_mb_per_sec to 0, the initial SSTable files 
generated by `compaction-stress write` are using the default 128MB size, which 
is inline with the typical flush size I saw on this kind of hardware using 
default cassandra.yaml configuration parameters), using 10GB of stress data 
generated by the blogpost data model 
[here|https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml],
 the overall compaction times with 1280MB max_sstable_size are: 7m16.456s, 
7m7.225s, 7m9.102s; the overall compaction times with 160MB max_sstable_size 
are: 9m16.715s, 9m28.146s, 9m7.192s.

Given these numbers, the average seconds to finish compaction with 1280MB 
max_sstable_size is 430.66, and the average seconds to finish compaction with 
160MB max_sstable_size is 557.33, which is already a 23% improvement.

I realize 10GB data is barely enough to test 1280MB sstable size as the data 
will only go from L0->L1, so the next run I'm going to use 100GB data size on 
this hardware (keeping everything else the same) and see how the numbers 
compare.

> Re-evaluate the default 160MB sstable_size_in_mb choice in LCS
> --
>
> Key: CASSANDRA-12591
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12591
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs
>
> There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
> the best max_sstable_size used by LeveledCompactionStrategy, and the 
> conclusion derived from that effort was to use 160MB as the most optimal size 
> for both throughput (i.e. the time spent on compaction, the smaller the 
> better) and the amount of bytes compacted (to avoid write amplification, the 
> less the better).
> However, when I read more into that test report (the short 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-5727?focusedCommentId=13722571=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13722571]
>  describing the tests), I realized it was conducted on a hardware with the 
> following configuration: "a single rackspace node with 2GB of ram." I'm not 
> sure if this was an ok hardware configuration for production Cassandra 
> deployment at that time (mid-2013), but it is definitely far lower from 
> today's hardware standard now.
> Given that we now have compaction-stress which is able to generate SSTables 
> based on user defined stress profile with user defined table schema and 
> compaction parameters (compatible to cassandra-stress), it would be a useful 
> effort to relook at this number using a more realistic hardware configuration 
> and see if 160MB is still the optimal choice. It might also impact our 
> perceived "practical" node density with LCS nodes if it turns out bigger 
> max_sstable_size actually works better as it will allow less number of 
> SSTables (and hence less level and less write amplification) per node with 
> bigger density.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-01 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12591:
-
Labels: lcs  (was: )

> Re-evaluate the default 160MB sstable_size_in_mb choice in LCS
> --
>
> Key: CASSANDRA-12591
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12591
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs
>
> There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
> the best max_sstable_size used by LeveledCompactionStrategy, and the 
> conclusion derived from that effort was to use 160MB as the most optimal size 
> for both throughput (i.e. the time spent on compaction, the smaller the 
> better) and the amount of bytes compacted (to avoid write amplification, the 
> less the better).
> However, when I read more into that test report, I realized it was conducted 
> on a hardware with the following configuration: "a single rackspace node with 
> 2GB of ram." I'm not sure if this was an ok hardware configuration for 
> production Cassandra deployment at that time (mid-2013), but it is definitely 
> far lower from today's hardware standard now.
> Given that we now have compaction-stress which is able to generate SSTables 
> based on user defined stress profile with user defined table schema and 
> compaction parameters (compatible to cassandra-stress), it would be a useful 
> effort to relook at this number using a more realistic hardware configuration 
> and see if 160MB is still the optimal choice. It might also impact our 
> perceived "practical" node density with LCS nodes if it turns out bigger 
> max_sstable_size actually works better as it will allow less number of 
> SSTables (and hence less level and less write amplification) per node with 
> bigger density.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-01 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12591?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12591:
-
Description: 
There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
the best max_sstable_size used by LeveledCompactionStrategy, and the conclusion 
derived from that effort was to use 160MB as the most optimal size for both 
throughput (i.e. the time spent on compaction, the smaller the better) and the 
amount of bytes compacted (to avoid write amplification, the less the better).

However, when I read more into that test report (the short 
[comment|https://issues.apache.org/jira/browse/CASSANDRA-5727?focusedCommentId=13722571=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13722571]
 describing the tests), I realized it was conducted on a hardware with the 
following configuration: "a single rackspace node with 2GB of ram." I'm not 
sure if this was an ok hardware configuration for production Cassandra 
deployment at that time (mid-2013), but it is definitely far lower from today's 
hardware standard now.

Given that we now have compaction-stress which is able to generate SSTables 
based on user defined stress profile with user defined table schema and 
compaction parameters (compatible to cassandra-stress), it would be a useful 
effort to relook at this number using a more realistic hardware configuration 
and see if 160MB is still the optimal choice. It might also impact our 
perceived "practical" node density with LCS nodes if it turns out bigger 
max_sstable_size actually works better as it will allow less number of SSTables 
(and hence less level and less write amplification) per node with bigger 
density.

  was:
There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
the best max_sstable_size used by LeveledCompactionStrategy, and the conclusion 
derived from that effort was to use 160MB as the most optimal size for both 
throughput (i.e. the time spent on compaction, the smaller the better) and the 
amount of bytes compacted (to avoid write amplification, the less the better).

However, when I read more into that test report, I realized it was conducted on 
a hardware with the following configuration: "a single rackspace node with 2GB 
of ram." I'm not sure if this was an ok hardware configuration for production 
Cassandra deployment at that time (mid-2013), but it is definitely far lower 
from today's hardware standard now.

Given that we now have compaction-stress which is able to generate SSTables 
based on user defined stress profile with user defined table schema and 
compaction parameters (compatible to cassandra-stress), it would be a useful 
effort to relook at this number using a more realistic hardware configuration 
and see if 160MB is still the optimal choice. It might also impact our 
perceived "practical" node density with LCS nodes if it turns out bigger 
max_sstable_size actually works better as it will allow less number of SSTables 
(and hence less level and less write amplification) per node with bigger 
density.


> Re-evaluate the default 160MB sstable_size_in_mb choice in LCS
> --
>
> Key: CASSANDRA-12591
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12591
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs
>
> There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
> the best max_sstable_size used by LeveledCompactionStrategy, and the 
> conclusion derived from that effort was to use 160MB as the most optimal size 
> for both throughput (i.e. the time spent on compaction, the smaller the 
> better) and the amount of bytes compacted (to avoid write amplification, the 
> less the better).
> However, when I read more into that test report (the short 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-5727?focusedCommentId=13722571=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13722571]
>  describing the tests), I realized it was conducted on a hardware with the 
> following configuration: "a single rackspace node with 2GB of ram." I'm not 
> sure if this was an ok hardware configuration for production Cassandra 
> deployment at that time (mid-2013), but it is definitely far lower from 
> today's hardware standard now.
> Given that we now have compaction-stress which is able to generate SSTables 
> based on user defined stress profile with user defined table schema and 
> compaction parameters (compatible to cassandra-stress), it would be a useful 
> effort to relook at this number using a more realistic hardware configuration 
> and see if 160MB is still the optimal choice. It might also impact our 
> perceived "practical" node density with 

[jira] [Created] (CASSANDRA-12591) Re-evaluate the default 160MB sstable_size_in_mb choice in LCS

2016-09-01 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12591:


 Summary: Re-evaluate the default 160MB sstable_size_in_mb choice 
in LCS
 Key: CASSANDRA-12591
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12591
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Wei Deng


There has been some effort from CASSANDRA-5727 in benchmarking and evaluating 
the best max_sstable_size used by LeveledCompactionStrategy, and the conclusion 
derived from that effort was to use 160MB as the most optimal size for both 
throughput (i.e. the time spent on compaction, the smaller the better) and the 
amount of bytes compacted (to avoid write amplification, the less the better).

However, when I read more into that test report, I realized it was conducted on 
a hardware with the following configuration: "a single rackspace node with 2GB 
of ram." I'm not sure if this was an ok hardware configuration for production 
Cassandra deployment at that time (mid-2013), but it is definitely far lower 
from today's hardware standard now.

Given that we now have compaction-stress which is able to generate SSTables 
based on user defined stress profile with user defined table schema and 
compaction parameters (compatible to cassandra-stress), it would be a useful 
effort to relook at this number using a more realistic hardware configuration 
and see if 160MB is still the optimal choice. It might also impact our 
perceived "practical" node density with LCS nodes if it turns out bigger 
max_sstable_size actually works better as it will allow less number of SSTables 
(and hence less level and less write amplification) per node with bigger 
density.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12040) If a level compaction fails due to no space it should schedule the next one

2016-08-31 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12040?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12040:
-
Labels: lcs  (was: )

>   If a level compaction fails due to no space it should schedule the next one
> -
>
> Key: CASSANDRA-12040
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12040
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: sankalp kohli
>Assignee: sankalp kohli
>Priority: Minor
>  Labels: lcs
> Fix For: 2.1.16, 2.2.8, 3.0.9, 3.8
>
> Attachments: CASSANDRA-12040_3.0.diff, CASSANDRA-12040_trunk.txt
>
>
> If a level compaction fails the space check, it aborts but next time the 
> compactions are scheduled it will attempt the same one. It should skip it and 
> go to the next so it can find smaller compactions to do.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11739) Cache key references might cause OOM on incremental repair

2016-08-31 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11739?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-11739:
-
Labels: lcs repair  (was: )

> Cache key references might cause OOM on incremental repair
> --
>
> Key: CASSANDRA-11739
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11739
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Paulo Motta
>Assignee: Paulo Motta
>  Labels: lcs, repair
> Fix For: 2.1.15, 2.2.7, 3.0.7, 3.7
>
> Attachments: heapdump.png
>
>
> We keep {{SSTableReader}} references for the duration of the repair to 
> anti-compact later, and their tidier keep references to cache keys to be 
> invalidated which are only cleaned up by GC after repair is finished. These 
> cache keys can accumulate while repair is being executed leading to OOM for 
> large tables/keyspaces.
> Heap dump attached.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11906) Unstable JVM due too many files when anticompacting big LCS tables

2016-08-31 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-11906:
-
Labels: lcs  (was: )

> Unstable JVM due too many files when anticompacting big LCS tables
> --
>
> Key: CASSANDRA-11906
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11906
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Stefano Ortolani
>Assignee: Sean McCarthy
>  Labels: lcs
> Fix For: 3.0.x
>
>
> I have recently moved from C* 2.1.x to C* 3.0.6. The setup is quite 
> heavy:
>   - 13 nodes with spinning disks
>   - ~120 GB of data per node
>   - 50% of CFs are LCS and have quite wide rows.
>   - 2/3 CFs with LCS have more than 200 SStables
> Incremental repairs do not seem to play really well with that.
> I have been running some tests node by node by using the -pr option:
> {code:xml}
> nodetool -h localhost repair -pr keyscheme
> {code}
> and to my surprise the whole process takes quite some time (1 hour
> minimum, 8 hours if I haven't been repairing for 5/6 days).
> Yesterday I tried to run the command with the -seq option so to 
> decrease the number of simultanoues compactions. After a while
> the node on which I was running the repair simply died during
> the anticompaction phase with the following
> exception in the logs.
> {code:xml}
> ERROR [metrics-graphite-reporter-1-thread-1] 2016-05-25 21:54:21,868 
> ScheduledReporter.java:119 - RuntimeException thrown from 
> GraphiteReporter#report. Exception was suppressed.
> java.lang.RuntimeException: Failed to list files in 
> /data/cassandra/data/keyschema/columnfamily-3996ce80b7ac11e48a9b6776bf484396
>   at 
> org.apache.cassandra.db.lifecycle.LogAwareFileLister.list(LogAwareFileLister.java:57)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.getFiles(LifecycleTransaction.java:547)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.Directories$SSTableLister.filter(Directories.java:691)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.Directories$SSTableLister.listFiles(Directories.java:662)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.Directories$TrueFilesSizeVisitor.(Directories.java:981)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.Directories.getTrueAllocatedSizeIn(Directories.java:893)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.Directories.trueSnapshotsSize(Directories.java:883) 
> ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.db.ColumnFamilyStore.trueSnapshotsSize(ColumnFamilyStore.java:2332)
>  ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.metrics.TableMetrics$32.getValue(TableMetrics.java:637) 
> ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> org.apache.cassandra.metrics.TableMetrics$32.getValue(TableMetrics.java:634) 
> ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at 
> com.codahale.metrics.graphite.GraphiteReporter.reportGauge(GraphiteReporter.java:281)
>  ~[metrics-graphite-3.1.0.jar:3.1.0]
>   at 
> com.codahale.metrics.graphite.GraphiteReporter.report(GraphiteReporter.java:158)
>  ~[metrics-graphite-3.1.0.jar:3.1.0]
>   at 
> com.codahale.metrics.ScheduledReporter.report(ScheduledReporter.java:162) 
> ~[metrics-core-3.1.0.jar:3.1.0]
>   at 
> com.codahale.metrics.ScheduledReporter$1.run(ScheduledReporter.java:117) 
> ~[metrics-core-3.1.0.jar:3.1.0]
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_91]
>   at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
> [na:1.8.0_91]
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  [na:1.8.0_91]
>   at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  [na:1.8.0_91]
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_91]
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_91]
>   at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> Caused by: java.lang.RuntimeException: java.nio.file.FileSystemException: 
> /data/cassandra/data/keyschema/columnfamily-3996ce80b7ac11e48a9b6776bf484396/ma_txn_anticompactionafterrepair_f20b50d0-22bd-11e6-970f-6f22464f4624.log:
>  Too many open files
>   at org.apache.cassandra.io.util.FileUtils.readLines(FileUtils.java:622) 
> ~[apache-cassandra-3.0.6.jar:3.0.6]
>   at java.util.stream.Collectors.lambda$toMap$58(Collectors.java:1321) 
> ~[na:1.8.0_91]
>   at 

[jira] [Updated] (CASSANDRA-12561) LCS compaction going into infinite loop due to non-existent sstables

2016-08-31 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12561?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12561:
-
Labels: lcs  (was: )

> LCS compaction going into infinite loop due to non-existent sstables
> 
>
> Key: CASSANDRA-12561
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12561
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Nimi Wariboko Jr.
>  Labels: lcs
>
> I believe this is related/similar to CASSANDRA-11373, but I'm running 3.5 and 
> I still have this issue.
> AFAICT, this happens when getCompactionCandidates in LeveledManifest.java 
> returns a candidate that does not exist on disk. 
> Eventually, all the compaction threads back up, garbage collections start 
> taking an upwards of 20 seconds and messages start being dropped.
> To get around this, I patched my instance with the following code in 
> LeveledManifest.java
> {code:java}
> Set removeCandidates = new HashSet<>();
> for (SSTableReader sstable : candidates)
> {
> if (!(new java.io.File(sstable.getFilename())).exists()) {
> removeCandidates.add(sstable);
> logger.warn("Not compating candidate {} because it does 
> not exist ({}).", sstable.getFilename(), sstable.openReason);
> }
> }
> candidates.removeAll(removeCandidates);
> if (candidates.size() < 2)
> return Collections.emptyList();
> else
> return candidates;
> {code}
> This just removes any candidate that doesn't exist on disk - however I'm not 
> sure what the side effects of this are.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11833) Bulk load mode for LCS

2016-08-24 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-11833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15435872#comment-15435872
 ] 

Wei Deng commented on CASSANDRA-11833:
--

I have done some rudimentary benchmark comparing a forced LCS major compaction 
and a regular "dumping everything into L0 with auto-compaction disabled and 
then enable auto-compaction and let all L0 SSTables to move up from one level 
to the next and so on". From the result I got I'm not convinced major 
compaction (at least in its current incarnation) is suitable for the "bulk 
load" mode we're talking about here.

The test platform is a GCE node with 4 cores and 16GB RAM. I know it's not a 
lot of hardware, but the observations from the benchmark led me to believe that 
we will easily see the same behavior on beefier machines. To have a fair 
comparison, I generated data via cassandra-stress and disabled auto-compaction 
and just leave flush running, so that I could accumulate 600+ L0 SSTables with 
83MB each (about 60GB total); I then performed a snapshot so that I always have 
the same data set for various experiments. The max_sstable_size (which is a 
table LCS parameter) is always set to 160MB in my tests. Since this is C* 3.x I 
don't have to worry about compression and just use the default schema from 
cassandra-stress that will create tables as uncompressed, which is easier for 
comparison. I then changed the concurrent_compactors to 4, and 
compaction_throughput_mb_per_sec to 0 in cassandra.yaml, so that all hardware 
on this machine is dedicated to running compactions.

With this configuration, I restarted the JVM and changed the schema to enable 
auto-compaction (while keeping all other activities completely idle, i.e. no 
read/write or any other external distractions that could consume CPU, IO or 
memory) and wait for the auto-compaction to finish; as a comparison for testing 
major compaction, I then restarted the JVM again and leave auto-compaction 
disabled this time, but run "nodetool compact keyspace1.standard1" to trigger 
the major compaction. In both cases, once the last compaction on the 
keyspace1.standard1 table finished, I calculated the wall-clock time as well as 
the aggregated time by adding up the time spent by each individual compaction 
session, and the aggregated written bytes. Here are the benchmark numbers from 
the two runs:

compaction type  | wall-clock time| aggregated time| 
aggregated written bytes

  
auto-compaction | 3.139 hours | 7.474 hours|
 244,166.164 MB
major compaction   |10.652 hours|   10.652 hours| 
60,960 MB

So as you can see, even though major compaction generates 4x smaller write (as 
it's only writing every single 160MB SSTable just once), it's taking way longer 
to finish, because the whole major compaction is just a single thread, while 
auto-compaction can leverage the 4 threads more effectively.

> Bulk load mode for LCS
> --
>
> Key: CASSANDRA-11833
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11833
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Jonathan Ellis
>  Labels: lcs
>
> We can currently avoid LCS write amplification on bulk load as follows:
> 1. Disable automatic compaction
> 2. Load all the data
> 3. Force major compaction
> 4. Re-enable automatic compaction
> It would be nice to automate this.  E.g. if we detect that L0 contains as 
> much data as the rest of our arena, skip bubbling sstables up and just major 
> compact automatically.  In a bulk load situation, this would result in 
> exponentially increasing major compaction sizes -- not as ideal as the manual 
> form with just a single one, but much better than our current behavior which 
> is not a good experience for users.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-24 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15435795#comment-15435795
 ] 

Wei Deng commented on CASSANDRA-12512:
--

Yep, the patch indeed fixed my problem. Now I can run both {{compaction-stress 
write}} and {{compaction-stress compact}} successfully.

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>Assignee: Yuki Morishita
> Fix For: 3.x
>
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@0ce59d338-1:~/cassandra-trunk$ ./tools/bin/compaction-stress write 
> -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> [UPDATE] It appears that {{compaction-stress compact}} fails on the same 
> assert but via a totally different code path. The stack trace is like the 
> following:
> {noformat}
> automaton@0ce59d338-1:~/cassandra-trunk$ ./tools/bin/compaction-stress 
> compact -d /tmp/compaction -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.db.SystemKeyspace.compile(SystemKeyspace.java:434)
>   at 
> org.apache.cassandra.db.SystemKeyspace.(SystemKeyspace.java:115)
>   at 
> org.apache.cassandra.stress.CompactionStress$Compaction.run(CompactionStress.java:213)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> (the last revision of the description had the wrong stack trace pasted and 
> I've corrected that.)
> As you can see this 2nd assert on {{compaction-stress compact}} is triggered 
> by SystemKeyspace class, so fix in StressProfile class is only able to solve 
> the assert problem for {{compaction-stress write}}, but not 
> {{compaction-stress compact}}.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12534) sstableverify tries to access commitlog directory and fails when running from a non-root/non-cassandra user

2016-08-24 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12534:


 Summary: sstableverify tries to access commitlog directory and 
fails when running from a non-root/non-cassandra user
 Key: CASSANDRA-12534
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12534
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Reporter: Wei Deng


Reproduced this against C* 3.0.8 code. See the following output (the 
{{automaton}} user running this command is not the user that runs the cassandra 
JVM process, so it cannot have access to the commit log directory):

{noformat}
automaton@0ce59d338-1:~$ sstableverify keyspace1 standard1
ERROR 19:38:37,674 Exiting due to error while processing commit log during 
initialization.
org.apache.cassandra.io.FSWriteError: java.nio.file.AccessDeniedException: 
/var/lib/cassandra/commitlog/CommitLog-6-1472067517651.log
at 
org.apache.cassandra.db.commitlog.CommitLogSegment.(CommitLogSegment.java:163)
 ~[cassandra-all-3.0.8.1293.jar:3.0.8.1293]
at 
org.apache.cassandra.db.commitlog.MemoryMappedSegment.(MemoryMappedSegment.java:47)
 ~[cassandra-all-3.0.8.1293.jar:3.0.8.1293]
at 
org.apache.cassandra.db.commitlog.CommitLogSegment.createSegment(CommitLogSegment.java:124)
 ~[cassandra-all-3.0.8.1293.jar:3.0.8.1293]
at 
org.apache.cassandra.db.commitlog.CommitLogSegmentManager$1.runMayThrow(CommitLogSegmentManager.java:122)
 ~[cassandra-all-3.0.8.1293.jar:3.0.8.1293]
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
[cassandra-all-3.0.8.1293.jar:3.0.8.1293]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66]
Caused by: java.nio.file.AccessDeniedException: 
/var/lib/cassandra/commitlog/CommitLog-6-1472067517651.log
at 
sun.nio.fs.UnixException.translateToIOException(UnixException.java:84) 
~[na:1.8.0_66]
at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) 
~[na:1.8.0_66]
at 
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) 
~[na:1.8.0_66]
at 
sun.nio.fs.UnixFileSystemProvider.newFileChannel(UnixFileSystemProvider.java:177)
 ~[na:1.8.0_66]
at java.nio.channels.FileChannel.open(FileChannel.java:287) 
~[na:1.8.0_66]
at java.nio.channels.FileChannel.open(FileChannel.java:335) 
~[na:1.8.0_66]
at 
org.apache.cassandra.db.commitlog.CommitLogSegment.(CommitLogSegment.java:158)
 ~[cassandra-all-3.0.8.1293.jar:3.0.8.1293]
... 5 common frames omitted
{noformat}

This is the same issue that should be helped by CASSANDRA-9054.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-7272) Add "Major" Compaction to LCS

2016-08-24 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7272?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15435388#comment-15435388
 ] 

Wei Deng commented on CASSANDRA-7272:
-

I've done some experiments regarding this feature on latest trunk (which will 
be 3.10 branch) and here are a few observations that were not clear from 
reading the comments in this JIRA, so I'm documenting them here in case anybody 
else may find useful later:

1. Once major compaction is triggered, it's going to include all existing 
SSTables from *all* levels regardless of where they're, so you don't have to 
run the offline {{sstablelevelreset}} first to manually send all SSTables to 
L0. If you watch debug.log, you will see a massive line of log entry like 
"CompactionTask.java:153 - Compacting x" that prints out all SSTables and 
their respective levels.

2. This major compaction is now carried out by one CompactionExecutor, so it 
will likely consume all CPU bandwidth of a single CPU core. This means the 
faster your CPU single-core performance is, the sooner you can get the major 
compaction done. However, there is no other parallelism whatsoever, as the same 
compaction thread will have to spit out all SSTables in sequence so that they 
can be on non-overlapping token ranges.

3. Before you start a major compaction, normally you want to run "nodetool 
disableautocompaction   && nodetool stop COMPACTION" so that no minor 
compaction is running for this table. When the table is set to not performing 
auto compaction, you can still start major compaction afterwards.

4. After you trigger the major compaction, you can still use "nodetool stop 
COMPACTION" to stop the running major compaction from continuing. And because 
of CASSANDRA-7066, it will revert all SSTables back to the previous status, 
which was nice.

5. Until the major compaction is completely finished, you won't see "nodetool 
cfstats" or debug.log entries to properly reflect the intermediate results 
(i.e. none of the newly created SSTables will be counted by cfstats or printed 
by debug.log) so there could be a major discrepancy in the total SSTable count 
and the number of SSTables in each level.

6. When the major compaction finishes, you may see a debug log entry showing 
that all newly generated SSTables are now compacted to L0, which is misleading, 
because there's actually an additional step taken by the major compaction to 
arrange them into different L1+ levels that is not reflected in the debug.log. 
It will follow the plan "write 10 files in L1, 100 files in L2 etc, starting 
from the lowest token (meaning L1 will not overlap at all with L2)". Since it's 
a simple metadata change (e.g. updating *-Statistics.db component) it can 
happen very quickly. Whether this approach will cause major write amplification 
and performance problem afterwards as mentioned by [~JiriHorky] still remains 
to be seen, and I believe some more tests will need to be done to prove it.

> Add "Major" Compaction to LCS 
> --
>
> Key: CASSANDRA-7272
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7272
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: T Jake Luciani
>Assignee: Marcus Eriksson
>Priority: Minor
>  Labels: compaction, docs-impacting, lcs
> Fix For: 2.2.0 beta 1
>
>
> LCS has a number of minor issues (maybe major depending on your perspective).
> LCS is primarily used for wide rows so for instance when you repair data in 
> LCS you end up with a copy of an entire repaired row in L0.  Over time if you 
> repair you end up with multiple copies of a row in L0 - L5.  This can make 
> predicting disk usage confusing.  
> Another issue is cleaning up tombstoned data.  If a tombstone lives in level 
> 1 and data for the cell lives in level 5 the data will not be reclaimed from 
> disk until the tombstone reaches level 5.
> I propose we add a "major" compaction for LCS that forces consolidation of 
> data to level 5 to address these.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8312) Use live sstables in snapshot repair if possible

2016-08-24 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8312?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8312:

Labels: lcs  (was: )

> Use live sstables in snapshot repair if possible
> 
>
> Key: CASSANDRA-8312
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8312
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jimmy Mårdell
>Assignee: Yuki Morishita
>Priority: Minor
>  Labels: lcs
> Fix For: 2.1.x
>
> Attachments: cassandra-2.0-8312-1.txt
>
>
> Snapshot repair can be very much slower than parallel repairs because of the 
> overhead of opening the SSTables in the snapshot. This is particular true 
> when using LCS, as you typically have many smaller SSTables then.
> I compared parallel and sequential repair on a small range on one of our 
> clusters (2*3 replicas). With parallel repair, this took 22 seconds. With 
> sequential repair (default in 2.0), the same range took 330 seconds! This is 
> an overhead of 330-22*6 = 198 seconds, just opening SSTables (there were 
> 1000+ sstables). Also, opening 1000 sstables for many smaller rangers surely 
> causes lots of memory churning.
> The idea would be to list the sstables in the snapshot, but use the 
> corresponding sstables in the live set if it's still available. For almost 
> all sstables, the original one should still exist.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12526) For LCS, single SSTable up-level is handled inefficiently

2016-08-23 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12526:
-
Description: 
I'm using the latest trunk (as of August 2016, which probably is going to be 
3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
inefficiency.

The test data is generated using cassandra-stress default parameters 
(keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
inserted partitions that will never merge in compactions, which is probably the 
worst kind of workload for LCS (however, I'll detail later why this scenario 
should not be ignored as a corner case; for now, let's just assume we still 
want to handle this scenario efficiently).

After the compaction test is done, I scrubbed debug.log for patterns that match 
 the "Compacted" summary so that I can see how long each individual compaction 
took and how many bytes they processed. The search pattern is like the 
following:

{noformat}
grep 'Compacted.*standard1' debug.log
{noformat}

Interestingly, I noticed a lot of the finished compactions are marked as having 
*only one* SSTable involved. With the workload mentioned above, the "single 
SSTable" compactions actually consist of the majority of all compactions (as 
shown below), so its efficiency can affect the overall compaction throughput 
quite a bit.

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | wc -l
243
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | grep ") 1 sstable" | wc -l
218
{noformat}

By looking at the code, it appears that there's a way to directly edit the 
level of a particular SSTable like the following:

{code}
sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 
targetLevel);
sstable.reloadSSTableMetadata();
{code}

To be exact, I summed up the time spent for these single-SSTable compactions 
(the total data size is 60GB) and found that if each compaction only needs to 
spend 100ms for only the metadata change (instead of the 10+ second they're 
doing now), it can already achieve 22.75% saving on total compaction time.

Compared to what we have now (reading the whole single-SSTable from old level 
and writing out the same single-SSTable at the new level), the only difference 
I could think of by using this approach is that the new SSTable will have the 
same file name (sequence number) as the old one's, which could break some 
assumptions on some other part of the code. However, not having to go through 
the full read/write IO, and not having to bear the overhead of cleaning up the 
old file, creating the new file, creating more churns in heap and file buffer, 
it seems the benefits outweigh the inconvenience. So I'd argue this JIRA 
belongs to LHF and should be made available in 3.0.x as well.

As mentioned in the 2nd paragraph, I'm also going to address why this kind of 
all-new-partition workload should not be ignored as a corner case. Basically, 
for the main use case of LCS where you need to frequently merge partitions to 
optimize read and eliminate tombstones and expired data sooner, LCS can be 
perfectly happy and efficiently perform the partition merge and tombstone 
elimination for a long time. However, as soon as the node becomes a bit 
unhealthy for various reasons (could be a bad disk so it's missing a whole 
bunch of mutations and need repair, could be the user chooses to ingest way 
more data than it usually takes and exceeds its capability, or god-forbidden, 
some DBA chooses to run offline sstablelevelreset), you will have to handle 
this kind of "all-new-partition with a lot of SSTables in L0" scenario, and 
once all L0 SSTables finally gets up-leveled to L1, you will likely see a lot 
of such single-SSTable compactions, which is the situation this JIRA is 
intended to address.

  was:
I'm using the latest trunk (as of August 2016, which probably is going to be 
3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
inefficiency.

The test data is generated using cassandra-stress default parameters 
(keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
inserted partitions that will never merge in compactions, which is probably the 
worst kind of workload for LCS (however, I'll detail later why this scenario 
should not be ignored as a corner case; for now, let's just assume we still 
want to handle this scenario efficiently).

After the compaction test is done, I scrubbed debug.log for patterns that match 
 the "Compacted" summary so that I can see how long each individual compaction 
took and how many bytes they processed. The search pattern is like the 
following:

{noformat}
grep 'Compacted.*standard1' debug.log
{noformat}

Interestingly, I noticed a lot of the finished compactions are marked as having 
*only one* SSTable 

[jira] [Updated] (CASSANDRA-12526) For LCS, single SSTable up-level is handled inefficiently

2016-08-23 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12526:
-
Description: 
I'm using the latest trunk (as of August 2016, which probably is going to be 
3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
inefficiency.

The test data is generated using cassandra-stress default parameters 
(keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
inserted partitions that will never merge in compactions, which is probably the 
worst kind of workload for LCS (however, I'll detail later why this scenario 
should not be ignored as a corner case; for now, let's just assume we still 
want to handle this scenario efficiently).

After the compaction test is done, I scrubbed debug.log for patterns that match 
 the "Compacted" summary so that I can see how long each individual compaction 
took and how many bytes they processed. The search pattern is like the 
following:

{noformat}
grep 'Compacted.*standard1' debug.log
{noformat}

Interestingly, I noticed a lot of the finished compactions are marked as having 
*only one* SSTable involved. With the workload mentioned above, the "single 
SSTable" compactions actually consist of the majority of all compactions (as 
shown below), so its efficiency can affect the overall compaction throughput 
quite a bit.

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | wc -l
243
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | grep ") 1 sstable" | wc -l
218
{noformat}

By looking at the code, it appears that there's a way to directly edit the 
level of a particular SSTable like the following:

{code}
sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 
targetLevel);
sstable.reloadSSTableMetadata();
{code}

Compared to what we have now (reading the whole single-SSTable from old level 
and writing out the same single-SSTable at the new level), the only difference 
I could think of by using this approach is that the new SSTable will have the 
same file name (sequence number) as the old one's, which could break some 
assumptions on some other part of the code. However, not having to go through 
the full read/write IO, and not having to bear the overhead of cleaning up the 
old file, creating the new file, creating more churns in heap and file buffer, 
it seems the benefits outweigh the inconvenience. So I'd argue this JIRA 
belongs to LHF and should be made available in 3.0.x as well.

As mentioned in the 2nd paragraph, I'm also going to address why this kind of 
all-new-partition workload should not be ignored as a corner case. Basically, 
for the main use case of LCS where you need to frequently merge partitions to 
optimize read and eliminate tombstones and expired data sooner, LCS can be 
perfectly happy and efficiently perform the partition merge and tombstone 
elimination for a long time. However, as soon as the node becomes a bit 
unhealthy for various reasons (could be a bad disk so it's missing a whole 
bunch of mutations and need repair, could be the user chooses to ingest way 
more data than it usually takes and exceeds its capability, or god-forbidden, 
some DBA chooses to run offline sstablelevelreset), you will have to handle 
this kind of "all-new-partition with a lot of SSTables in L0" scenario, and 
once all L0 SSTables finally gets up-leveled to L1, you will likely see a lot 
of such single-SSTable compactions, which is the situation this JIRA is 
intended to address.

  was:
I'm using the latest trunk (as of August 2016, which probably is going to be 
3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
inefficiency.

The test data is generated using cassandra-stress default parameters 
(keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
inserted partitions that will never merge in compactions, which is probably the 
worst kind of workload for LCS (however, I'll detail later why this scenario 
should not be ignored as a corner case; for now, let's just assume we still 
want to handle this scenario efficiently).

After the compaction test is done, I scrubbed debug.log for patterns that match 
 the "Compacted" summary so that I can see how long each individual compaction 
took and how many bytes they processed. The search pattern is like the 
following:

{noformat}
grep 'Compacted.*standard1' debug.log
{noformat}

Interestingly, I noticed a lot of the finished compactions are marked as having 
*only one* SSTable involved. With the workload mentioned above, the "single 
SSTable" compactions actually consist of the majority of all compactions (as 
shown below), so its efficiency can affect the overall compaction throughput 
quite a bit.

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 

[jira] [Created] (CASSANDRA-12526) For LCS, single SSTable up-level is handled inefficiently

2016-08-23 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12526:


 Summary: For LCS, single SSTable up-level is handled inefficiently
 Key: CASSANDRA-12526
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12526
 Project: Cassandra
  Issue Type: Bug
  Components: Compaction
Reporter: Wei Deng


I'm using the latest trunk (as of August 2016, which probably is going to be 
3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
inefficiency.

The test data is generated using cassandra-stress default parameters 
(keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
inserted partitions that will never merge in compactions, which is probably the 
worst kind of workload for LCS (however, I'll detail later why this scenario 
should not be ignored as a corner case; for now, let's just assume we still 
want to handle this scenario efficiently).

After the compaction test is done, I scrubbed debug.log for patterns that match 
 the "Compacted" summary so that I can see how long each individual compaction 
took and how many bytes they processed. The search pattern is like the 
following:

{noformat}
grep 'Compacted.*standard1' debug.log
{noformat}

Interestingly, I noticed a lot of the finished compactions are marked as having 
*only one* SSTable involved. With the workload mentioned above, the "single 
SSTable" compactions actually consist of the majority of all compactions (as 
shown below), so its efficiency can affect the overall compaction throughput 
quite a bit.

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | wc -l
243
automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
debug.log-test1 | grep ") 1 sstable" | wc -l
218
{noformat}

By looking at the code, it appears that there's a way to directly edit the 
level of a particular SSTable like the following:

{code}
sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 
targetLevel);
sstable.reloadSSTableMetadata();
{code}

Compared to what we have now (reading the whole single-SSTable from old level 
and writing out the same single-SSTable at the new level), the only difference 
I could think of by using this approach is that the new SSTable will have the 
same file name (sequence number) as the old one's, which could break some 
assumptions on some other part of the code. However, not having to go through 
the full read/write IO, and not having to bear the overhead of cleaning up the 
old file, creating the new file, creating more churns in heap and file buffer, 
it seems the benefits outweigh the inconvenience. So I'd argue this JIRA 
belongs to LHF and should be made available in 3.0.x as well.





--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12526) For LCS, single SSTable up-level is handled inefficiently

2016-08-23 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12526?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12526:
-
Labels: compaction lcs performance  (was: )

> For LCS, single SSTable up-level is handled inefficiently
> -
>
> Key: CASSANDRA-12526
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12526
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: compaction, lcs, performance
>
> I'm using the latest trunk (as of August 2016, which probably is going to be 
> 3.10) to run some experiments on LeveledCompactionStrategy and noticed this 
> inefficiency.
> The test data is generated using cassandra-stress default parameters 
> (keyspace1.standard1), so as you can imagine, it consists of a ton of newly 
> inserted partitions that will never merge in compactions, which is probably 
> the worst kind of workload for LCS (however, I'll detail later why this 
> scenario should not be ignored as a corner case; for now, let's just assume 
> we still want to handle this scenario efficiently).
> After the compaction test is done, I scrubbed debug.log for patterns that 
> match  the "Compacted" summary so that I can see how long each individual 
> compaction took and how many bytes they processed. The search pattern is like 
> the following:
> {noformat}
> grep 'Compacted.*standard1' debug.log
> {noformat}
> Interestingly, I noticed a lot of the finished compactions are marked as 
> having *only one* SSTable involved. With the workload mentioned above, the 
> "single SSTable" compactions actually consist of the majority of all 
> compactions (as shown below), so its efficiency can affect the overall 
> compaction throughput quite a bit.
> {noformat}
> automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
> debug.log-test1 | wc -l
> 243
> automaton@0ce59d338-1:~/cassandra-trunk/logs$ grep 'Compacted.*standard1' 
> debug.log-test1 | grep ") 1 sstable" | wc -l
> 218
> {noformat}
> By looking at the code, it appears that there's a way to directly edit the 
> level of a particular SSTable like the following:
> {code}
> sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 
> targetLevel);
> sstable.reloadSSTableMetadata();
> {code}
> Compared to what we have now (reading the whole single-SSTable from old level 
> and writing out the same single-SSTable at the new level), the only 
> difference I could think of by using this approach is that the new SSTable 
> will have the same file name (sequence number) as the old one's, which could 
> break some assumptions on some other part of the code. However, not having to 
> go through the full read/write IO, and not having to bear the overhead of 
> cleaning up the old file, creating the new file, creating more churns in heap 
> and file buffer, it seems the benefits outweigh the inconvenience. So I'd 
> argue this JIRA belongs to LHF and should be made available in 3.0.x as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-5342) ancestors are not cleared in SSTableMetadata after compactions are done and old SSTables are removed

2016-08-23 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5342?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-5342:

Labels: lcs  (was: )

> ancestors are not cleared in SSTableMetadata after compactions are done and 
> old SSTables are removed
> 
>
> Key: CASSANDRA-5342
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5342
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.1.3
>Reporter: Wei Zhu
>Assignee: Marcus Eriksson
>  Labels: lcs
> Fix For: 1.2.7
>
> Attachments: 0001-CASSANDRA-5342-wip-v2.patch, 
> 0001-CASSANDRA-5342-wip.patch, Screen Shot 2013-03-13 at 12.05.08 PM.png
>
>
> We are using LCS and have total of 38000 SSTables for one CF. During LCS, 
> there could be over a thousand SSTable involved. All those SSTable IDs are 
> stored in ancestors field of SSTableMetatdata for the new table. In our case, 
> it consumes more than 1G of heap memory for those field. Put it in 
> perspective, the ancestors consume 2 - 3 times more memory than bloomfilter 
> (fp = 0.1 by default) in LCS. 
> We should remove those ancestors from SSTableMetadata after the compaction is 
> finished and the old SSTable is removed. It  might be a big deal for Sized 
> Compaction since there are small number of SSTable involved. But it consumes 
> a lot of memory for LCS. 
> At least, we shouldn't load those ancestors to the memory during startup if 
> the files are removed. 
> I would love to contribute and provide patch. Please let me know how to 
> start. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12464) Investigate the potential improvement of parallelism on higher level compactions in LCS

2016-08-22 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12464:
-
Labels: lcs lhf performance  (was: lcs performance)

> Investigate the potential improvement of parallelism on higher level 
> compactions in LCS
> ---
>
> Key: CASSANDRA-12464
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12464
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs, lhf, performance
>
> According to LevelDB's design doc 
> [here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
> "A compaction merges the contents of the picked files to produce a sequence 
> of level-(L+1) files", it will "switch to producing a new level-(L+1) file 
> after the current output file has reached the target file size" (in our case 
> 160MB), it will also "switch to a new output file when the key range of the 
> current output file has grown enough to overlap more than ten level-(L+2) 
> files". This is to ensure "that a later compaction of a level-(L+1) file will 
> not pick up too much data from level-(L+2)."
> Our current code in LeveledCompactionStrategy doesn't implement this last 
> rule, but we might be able to quickly implement it and see how much a 
> compaction throughput improvement it can deliver. Potentially we can create a 
> scenario where a number of large L0 SSTables are present (e.g. 200GB after 
> switching from STCS) and let it to create thousands of L1 SSTables overflow, 
> and see how fast LCS can digest this much data from L1 and properly 
> upper-level them to completion.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-21 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15429576#comment-15429576
 ] 

Wei Deng edited comment on CASSANDRA-12512 at 8/21/16 7:14 PM:
---

Yep, after applying this 
[patch|https://issues.apache.org/jira/secure/attachment/12824146/0001-Replaced-using-CFMetaData.compile-in-cassandra-stres.patch],
 I was able to get {{compaction-stress write}} to work.

To apply the patch cleanly, the additional import packages needed to be 
manually inserted but it was trivial.


was (Author: weideng):
Yep, after applying this 
[patch|https://issues.apache.org/jira/secure/attachment/12824146/0001-Replaced-using-CFMetaData.compile-in-cassandra-stres.patch],
 I was able to get compaction-stress to work.

To apply the patch cleanly, the additional import packages needed to be 
manually inserted but it was trivial.

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@0ce59d338-1:~/cassandra-trunk$ ./tools/bin/compaction-stress write 
> -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> [UPDATE] It appears that {{compaction-stress compact}} fails on the same 
> assert but via a totally different code path. The stack trace is like the 
> following:
> {noformat}
> automaton@0ce59d338-1:~/cassandra-trunk$ ./tools/bin/compaction-stress 
> compact -d /tmp/compaction -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.db.SystemKeyspace.compile(SystemKeyspace.java:434)
>   at 
> org.apache.cassandra.db.SystemKeyspace.(SystemKeyspace.java:115)
>   at 
> org.apache.cassandra.stress.CompactionStress$Compaction.run(CompactionStress.java:213)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> (the last revision of the description had the wrong stack trace pasted and 
> I've corrected that.)
> As you can see this 2nd assert on {{compaction-stress compact}} is triggered 
> by SystemKeyspace class, so fix in StressProfile class is only able to solve 
> the assert problem for {{compaction-stress write}}, but not 
> {{compaction-stress compact}}.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-21 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12512:
-
Description: 
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk$ ./tools/bin/compaction-stress write -d 
/tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

[UPDATE] It appears that {{compaction-stress compact}} fails on the same assert 
but via a totally different code path. The stack trace is like the following:

{noformat}
automaton@0ce59d338-1:~/cassandra-trunk$ ./tools/bin/compaction-stress compact 
-d /tmp/compaction -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.db.SystemKeyspace.compile(SystemKeyspace.java:434)
at 
org.apache.cassandra.db.SystemKeyspace.(SystemKeyspace.java:115)
at 
org.apache.cassandra.stress.CompactionStress$Compaction.run(CompactionStress.java:213)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

(the last revision of the description had the wrong stack trace pasted and I've 
corrected that.)

As you can see this 2nd assert on {{compaction-stress compact}} is triggered by 
SystemKeyspace class, so fix in StressProfile class is only able to solve the 
assert problem for {{compaction-stress write}}, but not {{compaction-stress 
compact}}.

  was:
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

[UPDATE] It appears that {{compaction-stress compact}} fails on the same assert 
but via a totally different code path. The stack trace is like the following:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress compact -d /tmp/compaction -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.ExceptionInInitializerError
at 
org.apache.cassandra.cql3.QueryProcessor$InternalStateInstance.(QueryProcessor.java:138)
at 
org.apache.cassandra.cql3.QueryProcessor$InternalStateInstance.(QueryProcessor.java:132)
at 

[jira] [Commented] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15429593#comment-15429593
 ] 

Wei Deng commented on CASSANDRA-12512:
--

So the first problem with {{compaction-stress write}} was resolved by changing 
StressProfile code to avoid calling CFMetaData.compile(). However, the second 
problem goes deeper, as it is not trivial to replace CFMetaData.compile() used 
in SchemaKeyspace class.

[~krummas] Is it possible to avoid calling {{SystemKeyspace.finishStartup()}}?

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> [UPDATE] It appears that {{compaction-stress compact}} fails on the same 
> assert but via a totally different code path. The stack trace is like the 
> following:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress compact -d /tmp/compaction -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.ExceptionInInitializerError
>   at 
> org.apache.cassandra.cql3.QueryProcessor$InternalStateInstance.(QueryProcessor.java:138)
>   at 
> org.apache.cassandra.cql3.QueryProcessor$InternalStateInstance.(QueryProcessor.java:132)
>   at 
> org.apache.cassandra.cql3.QueryProcessor.internalQueryState(QueryProcessor.java:173)
>   at 
> org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal(QueryProcessor.java:353)
>   at 
> org.apache.cassandra.schema.SchemaKeyspace.saveSystemKeyspacesSchema(SchemaKeyspace.java:263)
>   at 
> org.apache.cassandra.db.SystemKeyspace.finishStartup(SystemKeyspace.java:494)
>   at 
> org.apache.cassandra.stress.CompactionStress$Compaction.run(CompactionStress.java:213)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> Caused by: java.lang.NullPointerException
>   at 
> org.apache.cassandra.service.ClientState.(ClientState.java:73)
>   ... 8 more
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12512:
-
Description: 
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

[UPDATE] It appears that {{compaction-stress compact}} fails on the same assert 
but via a totally different code path. The stack trace is like the following:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress compact -d /tmp/compaction -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.ExceptionInInitializerError
at 
org.apache.cassandra.cql3.QueryProcessor$InternalStateInstance.(QueryProcessor.java:138)
at 
org.apache.cassandra.cql3.QueryProcessor$InternalStateInstance.(QueryProcessor.java:132)
at 
org.apache.cassandra.cql3.QueryProcessor.internalQueryState(QueryProcessor.java:173)
at 
org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal(QueryProcessor.java:353)
at 
org.apache.cassandra.schema.SchemaKeyspace.saveSystemKeyspacesSchema(SchemaKeyspace.java:263)
at 
org.apache.cassandra.db.SystemKeyspace.finishStartup(SystemKeyspace.java:494)
at 
org.apache.cassandra.stress.CompactionStress$Compaction.run(CompactionStress.java:213)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
Caused by: java.lang.NullPointerException
at 
org.apache.cassandra.service.ClientState.(ClientState.java:73)
... 8 more
{noformat}

  was:
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}



> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is 

[jira] [Issue Comment Deleted] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12512:
-
Comment: was deleted

(was: [~blambov] Do you know if there's a quick workaround for this problem? 
Thanks.)

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12512:
-
Description: 
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}


  was:
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

Probably another code path not covered by CASSANDRA-8143? From the 
[comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
 it appears that this should no longer happen after the patch and I confirmed 
the patch is in the code I'm using.

To be exact for reproduce, I'm building from trunk with this commit hash (Aug 
20, 2016): 
https://github.com/apache/cassandra/tree/7eb6b997f216d9971499d877bfc74489c9a46681


> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> 

[jira] [Commented] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15429576#comment-15429576
 ] 

Wei Deng commented on CASSANDRA-12512:
--

Yep, after applying this 
[patch|https://issues.apache.org/jira/secure/attachment/12824146/0001-Replaced-using-CFMetaData.compile-in-cassandra-stres.patch],
 I was able to get compaction-stress to work.

To apply the patch cleanly, the additional import packages needed to be 
manually inserted but it was trivial.

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> Probably another code path not covered by CASSANDRA-8143? From the 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
>  it appears that this should no longer happen after the patch and I confirmed 
> the patch is in the code I'm using.
> To be exact for reproduce, I'm building from trunk with this commit hash (Aug 
> 20, 2016): 
> https://github.com/apache/cassandra/tree/7eb6b997f216d9971499d877bfc74489c9a46681



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15429575#comment-15429575
 ] 

Wei Deng commented on CASSANDRA-12512:
--

I think this is the same bug as discovered in CASSANDRA-12478, i.e. 
StressProfile is still using CFMetaData.compile().

I'm going to test the patch Denis provided in 12478 and report back if the 
problem is fixed.

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> Probably another code path not covered by CASSANDRA-8143? From the 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
>  it appears that this should no longer happen after the patch and I confirmed 
> the patch is in the code I'm using.
> To be exact for reproduce, I'm building from trunk with this commit hash (Aug 
> 20, 2016): 
> https://github.com/apache/cassandra/tree/7eb6b997f216d9971499d877bfc74489c9a46681



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12512:
-
Description: 
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

Probably another code path not covered by CASSANDRA-8143? From the 
[comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
 it appears that this should no longer happen after the patch and I confirmed 
the patch is in the code I'm using.

To be exact for reproduce, I'm building from trunk with this commit hash (Aug 
20, 2016): 
https://github.com/apache/cassandra/tree/7eb6b997f216d9971499d877bfc74489c9a46681

  was:
When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

Probably another code path not covered by CASSANDRA-8143? From the 
[comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
 it appears that this should no longer happen after the patch and I confirmed 
the patch is in the code I'm using.


> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> 

[jira] [Commented] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-12512?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15429525#comment-15429525
 ] 

Wei Deng commented on CASSANDRA-12512:
--

[~blambov] Do you know if there's a quick workaround for this problem? Thanks.

> compaction-stress: assertion error on accessing Schema.instance from 
> client-mode tool
> -
>
> Key: CASSANDRA-12512
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Wei Deng
>
> When I was trying the new compaction-stress tool from 3.10, I ran into the 
> following error:
> {noformat}
> automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
> ./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
> https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
>  -t 4
> java.lang.AssertionError: This assertion failure is probably due to accessing 
> Schema.instance from client-mode tools - See CASSANDRA-8143.
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
>   at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
>   at 
> org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
>   at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
>   at 
> org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
>   at 
> org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
>   at 
> org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
>   at 
> org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
>   at 
> org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
> {noformat}
> Probably another code path not covered by CASSANDRA-8143? From the 
> [comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
>  it appears that this should no longer happen after the patch and I confirmed 
> the patch is in the code I'm using.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12512) compaction-stress: assertion error on accessing Schema.instance from client-mode tool

2016-08-20 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12512:


 Summary: compaction-stress: assertion error on accessing 
Schema.instance from client-mode tool
 Key: CASSANDRA-12512
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12512
 Project: Cassandra
  Issue Type: Bug
Reporter: Wei Deng


When I was trying the new compaction-stress tool from 3.10, I ran into the 
following error:

{noformat}
automaton@wdengsummitsparkgoogle-0ce59d338-1:~/cassandra-trunk$ 
./tools/bin/compaction-stress write -d /tmp/compaction -g 5 -p 
https://gist.githubusercontent.com/tjake/8995058fed11d9921e31/raw/a9334d1090017bf546d003e271747351a40692ea/blogpost.yaml
 -t 4
java.lang.AssertionError: This assertion failure is probably due to accessing 
Schema.instance from client-mode tools - See CASSANDRA-8143.
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:288)
at org.apache.cassandra.config.CFMetaData.(CFMetaData.java:66)
at 
org.apache.cassandra.config.CFMetaData$Builder.build(CFMetaData.java:1332)
at org.apache.cassandra.config.CFMetaData.compile(CFMetaData.java:433)
at 
org.apache.cassandra.stress.StressProfile.init(StressProfile.java:174)
at 
org.apache.cassandra.stress.StressProfile.load(StressProfile.java:801)
at 
org.apache.cassandra.stress.CompactionStress.getStressProfile(CompactionStress.java:162)
at 
org.apache.cassandra.stress.CompactionStress$DataWriter.run(CompactionStress.java:289)
at 
org.apache.cassandra.stress.CompactionStress.main(CompactionStress.java:353)
{noformat}

Probably another code path not covered by CASSANDRA-8143? From the 
[comment|https://issues.apache.org/jira/browse/CASSANDRA-8143?focusedCommentId=14802787=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14802787]
 it appears that this should no longer happen after the patch and I confirmed 
the patch is in the code I'm using.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-6216) Level Compaction should persist last compacted key per level

2016-08-19 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6216?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15428713#comment-15428713
 ] 

Wei Deng commented on CASSANDRA-6216:
-

Isn't this already implemented so this should be closed with a proper fixVer 
number?

> Level Compaction should persist last compacted key per level
> 
>
> Key: CASSANDRA-6216
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6216
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: sankalp kohli
>Assignee: sankalp kohli
>Priority: Minor
>  Labels: compaction, lcs
> Attachments: JIRA-6216.diff
>
>
> Level compaction does not persist the last compacted key per level. This is 
> important for higher levels. 
> The sstables with higher token and in higher levels wont get a chance to 
> compact as the last compacted key will get reset after a restart.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-6323) Create new sstables in the highest possible level

2016-08-19 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-6323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-6323:

Labels: compaction lcs  (was: compaction)

> Create new sstables in the highest possible level
> -
>
> Key: CASSANDRA-6323
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6323
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Jonathan Ellis
>Assignee: Minh Do
>Priority: Minor
>  Labels: compaction, lcs
> Fix For: 2.1.x
>
>
> See PickLevelForMemTableOutput here: 
> https://github.com/google/leveldb/blob/master/db/version_set.cc#L493
> (Moving from CASSANDRA-5936)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-9935) Repair fails with RuntimeException

2016-08-18 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-9935?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-9935:

Labels: lcs  (was: )

> Repair fails with RuntimeException
> --
>
> Key: CASSANDRA-9935
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9935
> Project: Cassandra
>  Issue Type: Bug
> Environment: C* 2.1.8, Debian Wheezy
>Reporter: mlowicki
>Assignee: Paulo Motta
>  Labels: lcs
> Fix For: 2.1.15, 2.2.7, 3.0.6, 3.6
>
> Attachments: 9935.patch, db1.sync.lati.osa.cassandra.log, 
> db5.sync.lati.osa.cassandra.log, system.log.10.210.3.117, 
> system.log.10.210.3.221, system.log.10.210.3.230
>
>
> We had problems with slow repair in 2.1.7 (CASSANDRA-9702) but after upgrade 
> to 2.1.8 it started to work faster but now it fails with:
> {code}
> ...
> [2015-07-29 20:44:03,956] Repair session 23a811b0-3632-11e5-a93e-4963524a8bde 
> for range (-5474076923322749342,-5468600594078911162] finished
> [2015-07-29 20:44:03,957] Repair session 336f8740-3632-11e5-a93e-4963524a8bde 
> for range (-8631877858109464676,-8624040066373718932] finished
> [2015-07-29 20:44:03,957] Repair session 4ccd8430-3632-11e5-a93e-4963524a8bde 
> for range (-5372806541854279315,-5369354119480076785] finished
> [2015-07-29 20:44:03,957] Repair session 59f129f0-3632-11e5-a93e-4963524a8bde 
> for range (8166489034383821955,8168408930184216281] finished
> [2015-07-29 20:44:03,957] Repair session 6ae7a9a0-3632-11e5-a93e-4963524a8bde 
> for range (6084602890817326921,6088328703025510057] finished
> [2015-07-29 20:44:03,957] Repair session 8938e4a0-3632-11e5-a93e-4963524a8bde 
> for range (-781874602493000830,-781745173070807746] finished
> [2015-07-29 20:44:03,957] Repair command #4 finished
> error: nodetool failed, check server logs
> -- StackTrace --
> java.lang.RuntimeException: nodetool failed, check server logs
> at 
> org.apache.cassandra.tools.NodeTool$NodeToolCmd.run(NodeTool.java:290)
> at org.apache.cassandra.tools.NodeTool.main(NodeTool.java:202)
> {code}
> After running:
> {code}
> nodetool repair --partitioner-range --parallel --in-local-dc sync
> {code}
> Last records in logs regarding repair are:
> {code}
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 09ff9e40-3632-11e5-a93e-4963524a8bde for range 
> (-7695808664784761779,-7693529816291585568] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 17d8d860-3632-11e5-a93e-4963524a8bde for range 
> (806371695398849,8065203836608925992] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 23a811b0-3632-11e5-a93e-4963524a8bde for range 
> (-5474076923322749342,-5468600594078911162] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 336f8740-3632-11e5-a93e-4963524a8bde for range 
> (-8631877858109464676,-8624040066373718932] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 4ccd8430-3632-11e5-a93e-4963524a8bde for range 
> (-5372806541854279315,-5369354119480076785] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 59f129f0-3632-11e5-a93e-4963524a8bde for range 
> (8166489034383821955,8168408930184216281] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 6ae7a9a0-3632-11e5-a93e-4963524a8bde for range 
> (6084602890817326921,6088328703025510057] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 8938e4a0-3632-11e5-a93e-4963524a8bde for range 
> (-781874602493000830,-781745173070807746] finished
> {code}
> but a bit above I see (at least two times in attached log):
> {code}
> ERROR [Thread-173887] 2015-07-29 20:44:03,853 StorageService.java:2959 - 
> Repair session 1b07ea50-3608-11e5-a93e-4963524a8bde for range 
> (5765414319217852786,5781018794516851576] failed with error 
> org.apache.cassandra.exceptions.RepairException: [repair 
> #1b07ea50-3608-11e5-a93e-4963524a8bde on sync/entity_by_id2, 
> (5765414319217852786,5781018794516851576]] Validation failed in /10.195.15.162
> java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.RepairException: [repair 
> #1b07ea50-3608-11e5-a93e-4963524a8bde on sync/entity_by_id2, 
> (5765414319217852786,5781018794516851576]] Validation failed in /10.195.15.162
> at java.util.concurrent.FutureTask.report(FutureTask.java:122) 
> [na:1.7.0_80]
> at java.util.concurrent.FutureTask.get(FutureTask.java:188) 
> [na:1.7.0_80]
> at 
> org.apache.cassandra.service.StorageService$4.runMayThrow(StorageService.java:2950)
>  

[jira] [Created] (CASSANDRA-12498) Shorten the sstable log message as it unnecessarily contains the full path of a SSTable

2016-08-18 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12498:


 Summary: Shorten the sstable log message as it unnecessarily 
contains the full path of a SSTable
 Key: CASSANDRA-12498
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12498
 Project: Cassandra
  Issue Type: Improvement
  Components: Observability
Reporter: Wei Deng
Priority: Minor


There are a lot of places in debug.log where we print out the name of a 
SSTable. This is useful to look at the full path of a SSTable file when you're 
investigating individual SSTable. However, during compaction, we often see 32 
SSTables getting compacted at the same time, and the corresponding log line 
becomes very repetitive and hard to read as most of them are repeating the same 
first part of the file system path again and again, like the following:

{noformat}
DEBUG [CompactionExecutor:94] 2016-08-18 06:33:17,185  CompactionTask.java:146 
- Compacting (a5ca2f10-650d-11e6-95ef-a561ab3c45e8) 
[/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-200-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-201-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-16-big-Data.db:level=0,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-204-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-205-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-203-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-202-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-207-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-206-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-3-big-Data.db:level=0,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-208-big-Data.db:level=1,
 
/var/lib/cassandra/data/keyspace1/standard1-139cc441650d11e6a038bfe806276de2/mb-209-big-Data.db:level=1,
 ]
{noformat}

We should remove any text one level before ksName/cfName-UUID/ as it's very 
easy to get them from cassandra.yaml. For JBOD configuration where you have 
multiple data directories, keeping the one level before ksName/cfName-UUID 
should be adequate.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-5051) Allow automatic cleanup after gc_grace

2016-08-16 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423224#comment-15423224
 ] 

Wei Deng commented on CASSANDRA-5051:
-

+1 to [~jjirsa], from 2016. :-)

> Allow automatic cleanup after gc_grace
> --
>
> Key: CASSANDRA-5051
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5051
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Brandon Williams
>Assignee: Vijay
>  Labels: vnodes
> Fix For: 3.x
>
> Attachments: 0001-5051-v4.patch, 0001-5051-v6.patch, 
> 0001-5051-with-test-fixes.patch, 0001-CASSANDRA-5051.patch, 
> 0002-5051-remove-upgradesstable-v4.patch, 
> 0002-5051-remove-upgradesstable.patch, 0004-5051-additional-test-v4.patch, 
> 5051-v2.txt
>
>
> When using vnodes, after adding a new node you have to run cleanup on all the 
> machines, because you don't know which are affected and chances are it was 
> most if not all of them.  As an alternative to this intensive process, we 
> could allow cleanup during compaction if the data is older than gc_grace (or 
> perhaps some other time period since people tend to use gc_grace hacks to get 
> rid of tombstones.)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12114) Cassandra startup takes an hour because of N*N operation

2016-08-15 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12114:
-
Labels: lcs performance  (was: )

> Cassandra startup takes an hour because of N*N operation
> 
>
> Key: CASSANDRA-12114
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12114
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Tom van der Woerdt
>Assignee: Jeff Jirsa
>  Labels: lcs, performance
> Fix For: 3.0.9, 3.9
>
>
> (There's a previous version of this ticket, which was very wrong about the 
> actual cause. Original is quoted below)
> In java.org.cassandra.db.ColumnFamilyStore, the function scrubDataDirectories 
> loops over all sstables and then for each sstable it cleans temporary files 
> from its directory.
> Since there are many sstables in a directory, this ends up cleaning the same 
> directory many times.
> When using leveledcompactionstrategy on a data set that is ~4TB per node, you 
> can easily end up with 200k files.
> Add N and N, and we get a N*N operation (scrubDataDirectories) which ends up 
> taking an hour (or more).
> (At this point I should probably point out that no, I am not sure about that. 
> At all. But I do know this takes an hour and jstack blames this function)
> As promised, original ticket below :
> {quote}
> A Cassandra cluster of ours has nodes with up to 4TB of data, in a single 
> table using leveled compaction having 200k files. While upgrading from 2.2.6 
> to 3.0.7 we noticed that it took a while to restart a node. And with "a 
> while" I mean we measured it at more than 60 minutes.
> jstack shows something interesting :
> {code}
> "main" #1 prio=5 os_prio=0 tid=0x7f30db0ea400 nid=0xdb22 runnable 
> [0x7f30de122000]
>java.lang.Thread.State: RUNNABLE
> at java.io.UnixFileSystem.list(Native Method)
> at java.io.File.list(File.java:1122)
> at java.io.File.listFiles(File.java:1248)
> at 
> org.apache.cassandra.io.sstable.Descriptor.getTemporaryFiles(Descriptor.java:172)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.scrubDataDirectories(ColumnFamilyStore.java:599)
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:245)
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:557)
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:685)
> {code}
> Going by the source of File.listFiles, it puts every file in a directory into 
> an array and *then* applies the filter.
> This is actually a known Java issue from 1999: 
> http://bugs.java.com/view_bug.do?bug_id=4285834 -- their "solution" was to 
> introduce new APIs in JRE7. I guess that makes listFiles deprecated for 
> larger directories (like when using LeveledCompactionStrategy).
> tl;dr: because Cassandra uses java.io.File.listFiles, service startup can 
> take an hour for larger data sets.
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12464) Investigate the potential improvement of parallelism on higher level compactions in LCS

2016-08-15 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12464:
-
Description: 
According to LevelDB's design doc 
[here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
"A compaction merges the contents of the picked files to produce a sequence of 
level-(L+1) files", it will "switch to producing a new level-(L+1) file after 
the current output file has reached the target file size" (in our case 160MB), 
it will also "switch to a new output file when the key range of the current 
output file has grown enough to overlap more than ten level-(L+2) files". This 
is to ensure "that a later compaction of a level-(L+1) file will not pick up 
too much data from level-(L+2)."

Our current code in LeveledCompactionStrategy doesn't implement this last rule, 
but we might be able to quickly implement it and see how much a compaction 
throughput improvement it can deliver. Potentially we can create a scenario 
where a number of large L0 SSTables are present (e.g. 200GB after switching 
from STCS) and let it to create thousands of L1 SSTables overflow, and see how 
fast LCS can digest this much data from L1 and properly upper-level them to 
completion.

  was:
According to LevelDB's design doc 
[here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
"A compaction merges the contents of the picked files to produce a
sequence of level-(L+1) files", it will "switch to producing a new
level-(L+1) file after the current output file has reached the target
file size" (in our case 160MB), it will also "switch to a new output file when 
the key range of the current output file has grown enough to overlap more than 
ten level-(L+2) files". This is to ensure "that a later compaction
of a level-(L+1) file will not pick up too much data from level-(L+2)."

Our current code in LeveledCompactionStrategy doesn't implement this last rule, 
but we might be able to quickly implement it and see how much a compaction 
throughput improvement it can deliver. Potentially we can create a scenario 
where a number of large L0 SSTables are present (e.g. 200GB after switching 
from STCS) and let it to create thousands of L1 SSTables overflow, and see how 
fast LCS can digest this much data from L1 and properly upper-level them to 
completion.


> Investigate the potential improvement of parallelism on higher level 
> compactions in LCS
> ---
>
> Key: CASSANDRA-12464
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12464
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs, performance
>
> According to LevelDB's design doc 
> [here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
> "A compaction merges the contents of the picked files to produce a sequence 
> of level-(L+1) files", it will "switch to producing a new level-(L+1) file 
> after the current output file has reached the target file size" (in our case 
> 160MB), it will also "switch to a new output file when the key range of the 
> current output file has grown enough to overlap more than ten level-(L+2) 
> files". This is to ensure "that a later compaction of a level-(L+1) file will 
> not pick up too much data from level-(L+2)."
> Our current code in LeveledCompactionStrategy doesn't implement this last 
> rule, but we might be able to quickly implement it and see how much a 
> compaction throughput improvement it can deliver. Potentially we can create a 
> scenario where a number of large L0 SSTables are present (e.g. 200GB after 
> switching from STCS) and let it to create thousands of L1 SSTables overflow, 
> and see how fast LCS can digest this much data from L1 and properly 
> upper-level them to completion.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12464) Investigate the potential improvement of parallelism on higher level compactions in LCS

2016-08-15 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12464:


 Summary: Investigate the potential improvement of parallelism on 
higher level compactions in LCS
 Key: CASSANDRA-12464
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12464
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Wei Deng


According to LevelDB's design doc 
[here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
"A compaction merges the contents of the picked files to produce a
sequence of level-(L+1) files", it will "switch to producing a new
level-(L+1) file after the current output file has reached the target
file size" (in our case 160MB), it will also "switch to a new output file when 
the key range of the current output file has grown enough to overlap more than 
ten level-(L+2) files". This is to ensure "that a later compaction
of a level-(L+1) file will not pick up too much data from level-(L+2)."

Our current code in LeveledCompactionStrategy doesn't implement this last rule, 
but we might be able to quickly implement it and see how much a compaction 
throughput improvement it can deliver. Potentially we can create a scenario 
where a number of large L0 SSTables are present (e.g. 200GB after switching 
from STCS) and let it to create thousands of L1 SSTables overflow, and see how 
fast LCS can digest this much data from L1 and properly upper-level them to 
completion.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12464) Investigate the potential improvement of parallelism on higher level compactions in LCS

2016-08-15 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12464:
-
Labels: lcs performance  (was: )

> Investigate the potential improvement of parallelism on higher level 
> compactions in LCS
> ---
>
> Key: CASSANDRA-12464
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12464
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>  Labels: lcs, performance
>
> According to LevelDB's design doc 
> [here|https://github.com/google/leveldb/blob/master/doc/impl.html#L115-L116], 
> "A compaction merges the contents of the picked files to produce a
> sequence of level-(L+1) files", it will "switch to producing a new
> level-(L+1) file after the current output file has reached the target
> file size" (in our case 160MB), it will also "switch to a new output file 
> when the key range of the current output file has grown enough to overlap 
> more than ten level-(L+2) files". This is to ensure "that a later compaction
> of a level-(L+1) file will not pick up too much data from level-(L+2)."
> Our current code in LeveledCompactionStrategy doesn't implement this last 
> rule, but we might be able to quickly implement it and see how much a 
> compaction throughput improvement it can deliver. Potentially we can create a 
> scenario where a number of large L0 SSTables are present (e.g. 200GB after 
> switching from STCS) and let it to create thousands of L1 SSTables overflow, 
> and see how fast LCS can digest this much data from L1 and properly 
> upper-level them to completion.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10540) RangeAwareCompaction

2016-08-15 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-10540:
-
Labels: compaction lcs vnodes  (was: )

> RangeAwareCompaction
> 
>
> Key: CASSANDRA-10540
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10540
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>  Labels: compaction, lcs, vnodes
> Fix For: 3.x
>
>
> Broken out from CASSANDRA-6696, we should split sstables based on ranges 
> during compaction.
> Requirements;
> * dont create tiny sstables - keep them bunched together until a single vnode 
> is big enough (configurable how big that is)
> * make it possible to run existing compaction strategies on the per-range 
> sstables
> We should probably add a global compaction strategy parameter that states 
> whether this should be enabled or not.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10810) Make rebuild operations resumable

2016-08-15 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-10810?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15421865#comment-15421865
 ] 

Wei Deng commented on CASSANDRA-10810:
--

[~pauloricardomg] What's the doc impact? Do we simply run "nodetool bootstrap 
resume" to resume from interrupted rebuild session, or we use "nodetool rebuild 
resume"?

> Make rebuild operations resumable
> -
>
> Key: CASSANDRA-10810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10810
> Project: Cassandra
>  Issue Type: Wish
>  Components: Streaming and Messaging
>Reporter: Jeremy Hanna
>Assignee: Kaide Mu
>
> Related to CASSANDRA-8942, now that we can resume bootstrap operations, this 
> could also be possible with rebuild operations, such as when you bootstrap 
> new nodes in a completely new datacenter in two steps.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-9588) Make sstableofflinerelevel print stats before relevel

2016-08-14 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-9588?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-9588:

Labels: lcs lhf  (was: lhf)

> Make sstableofflinerelevel print stats before relevel
> -
>
> Key: CASSANDRA-9588
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9588
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Tools
>Reporter: Jens Rantil
>Assignee: Marcus Eriksson
>Priority: Trivial
>  Labels: lcs, lhf
> Fix For: 3.6
>
>   Original Estimate: 1h
>  Remaining Estimate: 1h
>
> The current version of sstableofflinerelevel prints the new level hierarchy. 
> While "nodetool cfstats ..." will tell the current hierarchy it would be nice 
> to have "sstableofflinerelevel" output the current level histograms for easy 
> comparison of what changes will be made. Especially since 
> sstableofflinerelevel needs to run when node isn't running and "nodetool 
> cfstats ..." doesn't work because of that.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-5271) Create tool to drop sstables to level 0

2016-08-14 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5271?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-5271:

Labels: lcs tools  (was: las tools)

> Create tool to drop sstables to level 0
> ---
>
> Key: CASSANDRA-5271
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5271
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Trivial
>  Labels: lcs, tools
> Fix For: 2.0 beta 1
>
> Attachments: 0001-Add-tool-to-drop-sstables-back-to-level-0-v2.patch, 
> 0001-Add-tool-to-drop-sstables-back-to-level-0-v3.patch, 
> 0001-Add-tool-to-drop-sstables-back-to-level-0.patch
>
>
> after CASSANDRA-4872 we need a way to reset all sstables to level 0, 
> previously we did this by removing the .json file from the data-directory.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-5271) Create tool to drop sstables to level 0

2016-08-14 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-5271?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-5271:

Labels: las tools  (was: tools)

> Create tool to drop sstables to level 0
> ---
>
> Key: CASSANDRA-5271
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5271
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Trivial
>  Labels: lcs, tools
> Fix For: 2.0 beta 1
>
> Attachments: 0001-Add-tool-to-drop-sstables-back-to-level-0-v2.patch, 
> 0001-Add-tool-to-drop-sstables-back-to-level-0-v3.patch, 
> 0001-Add-tool-to-drop-sstables-back-to-level-0.patch
>
>
> after CASSANDRA-4872 we need a way to reset all sstables to level 0, 
> previously we did this by removing the .json file from the data-directory.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12366) Fix compaction throttle

2016-08-13 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12366?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12366:
-
Labels: compaction performance  (was: )

> Fix compaction throttle
> ---
>
> Key: CASSANDRA-12366
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12366
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: T Jake Luciani
>Assignee: T Jake Luciani
>  Labels: compaction, performance
> Fix For: 3.10
>
>
> Compaction throttling is broken in the following ways:
>   * It throttles bytes read after being decompressed
>   * Compaction creates multiple scanners which share the rate limiter causing 
> too much throttling
>   * It bears no resemblance to the reported compaction time remaining 
> calculation (Bytes of source sstables processed since start of compaction)
> To fix this we need to simplify the throttling to be only at the 
> CompactionIterator level.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11035) Use cardinality estimation to pick better compaction candidates for STCS (SizeTieredCompactionStrategy)

2016-08-12 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-11035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15418489#comment-15418489
 ] 

Wei Deng commented on CASSANDRA-11035:
--

Is there plan to make this available in trunk?

> Use cardinality estimation to pick better compaction candidates for STCS 
> (SizeTieredCompactionStrategy)
> ---
>
> Key: CASSANDRA-11035
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11035
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>Assignee: Marcus Eriksson
>
> This was initially mentioned in this blog post 
> http://www.datastax.com/dev/blog/improving-compaction-in-cassandra-with-cardinality-estimation
>  but I couldn't find any existing JIRA for it. As stated by [~jbellis], 
> "Potentially even more useful would be using cardinality estimation to pick 
> better compaction candidates. Instead of blindly merging sstables of a 
> similar size a la SizeTieredCompactionStrategy." The L0 STCS in LCS should 
> benefit as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11865) Improve compaction logging details

2016-08-05 Thread Wei Deng (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-11865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15410084#comment-15410084
 ] 

Wei Deng commented on CASSANDRA-11865:
--

[~tjake] and [~carlyeks], I also noticed many of the logger messages in 
o.a.c.db.compaction.CompactionManager are missing references to keyspace name 
and table name, while most of the methods already have the ColumnFamilyStore 
object in the context, so they can be easily added as cfs.keyspace.getName(), 
cfs.name in the logger calls.

For example, [this log 
line|https://github.com/apache/cassandra/blob/143d16961a67cc9d5608605ee2561253de629d2c/src/java/org/apache/cassandra/db/compaction/CompactionManager.java#L162]
 could have printed out the keyspace name and table name to allow people to 
understand exactly what table has autocompaction disabled.

Could you add reference to keyspace name and table name whenever possible in 
our log messages, as part of your effort for this JIRA?



> Improve compaction logging details
> --
>
> Key: CASSANDRA-11865
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11865
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Compaction
>Reporter: T Jake Luciani
>Assignee: Carl Yeksigian
>
> I'd like to see per compaction entry:
>   * Partitions processed
>   * Rows processed
>   * Partition merge stats
>   * If a wide row was detected
>   * The partition min/max/avg size
>   * The min/max/avg row count across partitions
> Anything else [~krummas]?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-12388) For LeveledCompactionStrategy, provide a JMX interface to trigger printing out LeveledManifest

2016-08-05 Thread Wei Deng (JIRA)
Wei Deng created CASSANDRA-12388:


 Summary: For LeveledCompactionStrategy, provide a JMX interface to 
trigger printing out LeveledManifest
 Key: CASSANDRA-12388
 URL: https://issues.apache.org/jira/browse/CASSANDRA-12388
 Project: Cassandra
  Issue Type: Improvement
  Components: Compaction
Reporter: Wei Deng
Priority: Minor


It appears that we have a private toString() method in LeveledManifest, but 
it's only used from within the class, even though it has capability of printing 
all SSTables on all levels. We used to be able to get this information from the 
manifest file in data directory, but after CASSANDRA-4872 this is no longer 
available. It will be useful for troubleshooting if we can have a JMX MBean 
method to trigger printing out the full generations list of a particular 
manifest instance.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-12388) For LeveledCompactionStrategy, provide a JMX interface to trigger printing out LeveledManifest

2016-08-05 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-12388?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-12388:
-
Description: For LCS, it appears that we have a private toString() method 
in LeveledManifest, but it's only used from within the class, even though it 
has capability of printing all SSTables on all levels. We used to be able to 
get this information from the manifest file in data directory, but after 
CASSANDRA-4872 this is no longer available. It will be useful for 
troubleshooting if we can have a JMX MBean method to trigger printing out the 
full generations list of a particular manifest instance.  (was: It appears that 
we have a private toString() method in LeveledManifest, but it's only used from 
within the class, even though it has capability of printing all SSTables on all 
levels. We used to be able to get this information from the manifest file in 
data directory, but after CASSANDRA-4872 this is no longer available. It will 
be useful for troubleshooting if we can have a JMX MBean method to trigger 
printing out the full generations list of a particular manifest instance.)

> For LeveledCompactionStrategy, provide a JMX interface to trigger printing 
> out LeveledManifest
> --
>
> Key: CASSANDRA-12388
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12388
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Wei Deng
>Priority: Minor
>  Labels: lcs
>
> For LCS, it appears that we have a private toString() method in 
> LeveledManifest, but it's only used from within the class, even though it has 
> capability of printing all SSTables on all levels. We used to be able to get 
> this information from the manifest file in data directory, but after 
> CASSANDRA-4872 this is no longer available. It will be useful for 
> troubleshooting if we can have a JMX MBean method to trigger printing out the 
> full generations list of a particular manifest instance.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10692) Don't remove level info when doing upgradesstables

2016-08-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10692?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-10692:
-
Labels: lcs  (was: )

> Don't remove level info when doing upgradesstables
> --
>
> Key: CASSANDRA-10692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>  Labels: lcs
> Fix For: 2.1.12, 2.2.4
>
>
> Seems we blow away the level info when doing upgradesstables. Introduced in  
> CASSANDRA-8004



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10422) Avoid anticompaction when doing subrange repair

2016-08-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10422?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-10422:
-
Labels: compaction repair  (was: )

> Avoid anticompaction when doing subrange repair
> ---
>
> Key: CASSANDRA-10422
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10422
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Marcus Eriksson
>Assignee: Ariel Weisberg
>  Labels: compaction, repair
> Fix For: 2.1.12, 2.2.4, 3.0.1, 3.1
>
>
> If we do split the owned range in say 1000 parts, and then do one repair 
> each, we could potentially anticompact every sstable 1000 times (ie, we 
> anticompact the repaired range out 1000 times). We should avoid 
> anticompacting at all in these cases.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8988) Optimise IntervalTree

2016-08-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8988?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8988:

Labels: compaction performance  (was: performance)

> Optimise IntervalTree
> -
>
> Key: CASSANDRA-8988
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8988
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Benedict
>Assignee: Benedict
>Priority: Minor
>  Labels: compaction, performance
> Fix For: 2.2.0 beta 1
>
> Attachments: 8988.txt
>
>
> We perform a lot of unnecessary comparisons in 
> IntervalTree.IntervalNode.searchInternal.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8920) Optimise sequential overlap visitation for checking tombstone retention in compaction

2016-08-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8920:

Labels: compaction lcs performance  (was: performance)

> Optimise sequential overlap visitation for checking tombstone retention in 
> compaction
> -
>
> Key: CASSANDRA-8920
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8920
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Benedict
>Assignee: Benedict
>Priority: Minor
>  Labels: compaction, lcs, performance
> Fix For: 2.2.0 beta 1
>
> Attachments: 8920.txt
>
>
> The IntervalTree only maps partition keys. Since a majority of users deploy a 
> hashed partitioner the work is mostly wasted, since they will be evenly 
> distributed across the full token range owned by the node - and in some cases 
> it is a significant amount of work. We can perform a corroboration against 
> the file bounds if we get a BF match as a sanity check if we like, but 
> performing an IntervalTree search is significantly more expensive (esp. once 
> murmur hash calculation memoization goes mainstream).
> In LCS, the keys are bounded, to it might appear that it would help, but in 
> this scenario we only compact against like bounds, so again it is not helpful.
> With a ByteOrderedPartitioner it could potentially be of use, but this is 
> sufficiently rare to not optimise for IMO.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8915) Improve MergeIterator performance

2016-08-03 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8915?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8915:

Labels: compaction lcs performance  (was: compaction performance)

> Improve MergeIterator performance
> -
>
> Key: CASSANDRA-8915
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8915
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Branimir Lambov
>Assignee: Branimir Lambov
>Priority: Minor
>  Labels: compaction, lcs, performance
> Fix For: 3.0 alpha 1
>
>  Time Spent: 0.2h
>  Remaining Estimate: 0h
>
> The implementation of {{MergeIterator}} uses a priority queue and applies a 
> pair of {{poll}}+{{add}} operations for every item in the resulting sequence. 
> This is quite inefficient as {{poll}} necessarily applies at least {{log N}} 
> comparisons (up to {{2log N}}), and {{add}} often requires another {{log N}}, 
> for example in the case where the inputs largely don't overlap (where {{N}} 
> is the number of iterators being merged).
> This can easily be replaced with a simple custom structure that can perform 
> replacement of the top of the queue in a single step, which will very often 
> complete after a couple of comparisons and in the worst case scenarios will 
> match the complexity of the current implementation.
> This should significantly improve merge performance for iterators with 
> limited overlap (e.g. levelled compaction).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8434) L0 should have a separate configurable bloom filter false positive ratio

2016-07-22 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8434?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8434:

Labels: las  (was: )

> L0 should have a separate configurable bloom filter false positive ratio
> 
>
> Key: CASSANDRA-8434
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8434
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Benedict
>  Labels: lcs
> Fix For: 2.1.x
>
>
> In follow up to CASSANDRA-5371. We now perform size-tiered file selection for 
> compaction if L0 gets too far behind, however as far as I can tell we stick 
> with the CF configured false positive ratio, likely inflating substantially 
> the number of files we visit on average until L0 is cleaned up. Having a a 
> different bf fp for L0 would solve this problem without introducing any 
> significant burden when L0 is not overloaded.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8434) L0 should have a separate configurable bloom filter false positive ratio

2016-07-22 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-8434?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-8434:

Labels: lcs  (was: las)

> L0 should have a separate configurable bloom filter false positive ratio
> 
>
> Key: CASSANDRA-8434
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8434
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Benedict
>  Labels: lcs
> Fix For: 2.1.x
>
>
> In follow up to CASSANDRA-5371. We now perform size-tiered file selection for 
> compaction if L0 gets too far behind, however as far as I can tell we stick 
> with the CF configured false positive ratio, likely inflating substantially 
> the number of files we visit on average until L0 is cleaned up. Having a a 
> different bf fp for L0 would solve this problem without introducing any 
> significant burden when L0 is not overloaded.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-4255) concurrent modif ex when repair is run on LCS

2016-07-21 Thread Wei Deng (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-4255?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wei Deng updated CASSANDRA-4255:

Labels: compaction lcs  (was: compaction)

> concurrent modif ex when repair is run on LCS
> -
>
> Key: CASSANDRA-4255
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4255
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.0.0
>Reporter: Jackson Chung
>Assignee: Jonathan Ellis
>Priority: Minor
>  Labels: compaction, lcs
> Fix For: 1.0.11, 1.1.1
>
> Attachments: 4255.txt
>
>
> came across this, will try to figure a way to systematically reprod this. But 
> the problem is the sstable list in the manifest is changing as the repair is 
> triggered:
> {panel}
> Exception in thread "main" java.util.ConcurrentModificationException 
>  at java.util.AbstractList$Itr.checkForComodification(Unknown Source)
>  at java.util.AbstractList$Itr.next(Unknown Source)
>  at org.apache.cassandra.io.sstable.SSTable.getTotalBytes(SSTable.java:250)
>  at 
> org.apache.cassandra.db.compaction.LeveledManifest.getEstimatedTasks(LeveledManifest.java:435)
>  at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getEstimatedRemainingTasks(LeveledCompactionStrategy.java:128)
>  at 
> org.apache.cassandra.db.compaction.CompactionManager.getPendingTasks(CompactionManager.java:1063)
>  at sun.reflect.GeneratedMethodAccessor73.invoke(Unknown Source)
>  at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
>  at java.lang.reflect.Method.invoke(Unknown Source)
>  at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(Unknown Source)
>  at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(Unknown Source)
>  at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(Unknown Source)
>  at com.sun.jmx.mbeanserver.PerInterface.getAttribute(Unknown Source)
>  at com.sun.jmx.mbeanserver.MBeanSupport.getAttribute(Unknown Source)
>  at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttribute(Unknown 
> Source)
>  at com.sun.jmx.mbeanserver.JmxMBeanServer.getAttribute(Unknown Source)
>  at javax.management.remote.rmi.RMIConnectionImpl.doOperation(Unknown Source)
>  at javax.management.remote.rmi.RMIConnectionImpl.access$200(Unknown Source)
>  at 
> javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(Unknown 
> Source)
>  at 
> javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(Unknown 
> Source)
>  at javax.management.remote.rmi.RMIConnectionImpl.getAttribute(Unknown Source)
>  at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source)
>  at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
>  at java.lang.reflect.Method.invoke(Unknown Source)
>  at sun.rmi.server.UnicastServerRef.dispatch(Unknown Source)
>  at sun.rmi.transport.Transport$1.run(Unknown Source)
>  at java.security.AccessController.doPrivileged(Native Method)
>  at sun.rmi.transport.Transport.serviceCall(Unknown Source)
>  at sun.rmi.transport.tcp.TCPTransport.handleMessages(Unknown Source)
>  at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(Unknown Source)
>  at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(Unknown Source)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>  at java.lang.Thread.run(Unknown Source)
> {panel}
> maybe we could change the list to a copyOnArrayList? just a suggestion, 
> haven't investigated much yet:
> {code:title=LeveledManifest.java}
> generations[i] = new ArrayList();
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


  1   2   3   4   >