[jira] [Comment Edited] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159482#comment-17159482 ] Caleb Rackliffe edited comment on CASSANDRA-15907 at 7/17/20, 5:38 AM: --- bq. the main advantage of this approach is that in the absence of conflicts nothing needs to be cached Even though partition-restricted queries without digest mismatches will skip {{DataResolver}} entirely, this still helps in the case where we have a mismatch, but start with a large number of conflict-free rows, correct? If so, I think this is a benefit we should consider, given how likely we are to be dealing with partition-restricted queries. bq. Also we could also just let it unbounded to minimize the number of RFP queries The one thing that makes me a little uneasy about this is the extra logic we need to enforce the "target cache size". I propose we avoid that, simply leave the guardrails we've already got in place to avoid catastrophe (and excessive RFP queries), and see if that means we can simplify what remains (like having to clear the {{contents}} array list between batches). I'll try this and see how it looks and pull it into the main 3.0 branch if it works. Aside from that, I think the only remaining question would be verifying the safety of [aggressively clearing|https://github.com/apache/cassandra/pull/659/commits/30b8f4bebd95b3520b637d6d25d6bc16cb4d81a2] {{responses}}. was (Author: maedhroz): bq. the main advantage of this approach is that in the absence of conflicts nothing needs to be cached Even though partition-restricted queries without digest mismatches will skip {{DataResolver}} entirely, this still helps in the case where we have a mismatch, but start with a large number of conflict-free rows, correct? If so, I think this is a benefit we can't ignore, given how likely we are to be dealing with partition-restricted queries. bq. Also we could also just let it unbounded to minimize the number of RFP queries The one thing that makes me a little uneasy about this is the extra logic we need to enforce the "target cache size". I propose we avoid that, simply leave the guardrails we've already got in place to avoid catastrophe (and excessive RFP queries), and see if that means we can simplify what remains (like having to clear the {{contents}} array list between batches). I'll try this and see how it looks and pull it into the main 3.0 branch if it works. Aside from that, I think the only remaining question would be verifying the safety of [aggressively clearing|https://github.com/apache/cassandra/pull/659/commits/30b8f4bebd95b3520b637d6d25d6bc16cb4d81a2] {{responses}}. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 4h 20m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtectio
[jira] [Comment Edited] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159688#comment-17159688 ] Caleb Rackliffe edited comment on CASSANDRA-15234 at 7/17/20, 5:32 AM: --- [~e.dimitrova] [~benedict] I've posted [a rough sketch|https://github.com/maedhroz/cassandra/commit/49e83c70eba3357978d1081ecf500bbbdee960d8] of what a fairly aggressive application of option grouping would look like for {{cassandra.yaml}}. Hopefully that helps keep the conversation moving forward. The biggest change I made to the existing structure was placing options for the same component in the same group, rather than having "commonly used" and "advanced" options (in some cases) in totally different parts of the file. was (Author: maedhroz): [~e.dimitrova] [~benedict] I've posted [a rough sketch|https://github.com/maedhroz/cassandra/commit/49e83c70eba3357978d1081ecf500bbbdee960d8] of what a fairly aggressive application of option grouping would look like for {{cassandra.yaml}}. Hopefully that helps keep the conversation moving forward... > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for either > their old or new definition e.g. {{KiB/s}}, or we could support them and > simply log the value in bytes/s. -- 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-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159688#comment-17159688 ] Caleb Rackliffe commented on CASSANDRA-15234: - [~e.dimitrova] [~benedict] I've posted [a rough sketch|https://github.com/maedhroz/cassandra/commit/49e83c70eba3357978d1081ecf500bbbdee960d8] of what a fairly aggressive application of option grouping would look like for {{cassandra.yaml}}. Hopefully that helps keep the conversation moving forward... > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for either > their old or new definition e.g. {{KiB/s}}, or we could support them and > simply log the value in bytes/s. -- 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-15955) java.nio.file.AccessDeniedException
rafi saeputra created CASSANDRA-15955: - Summary: java.nio.file.AccessDeniedException Key: CASSANDRA-15955 URL: https://issues.apache.org/jira/browse/CASSANDRA-15955 Project: Cassandra Issue Type: Bug Components: Local/Startup and Shutdown Reporter: rafi saeputra when i installed cassandra with binnary tarball i got an error log like this ERROR [COMMIT-LOG-ALLOCATOR] 2020-07-17 11:47:33,712 JVMStabilityInspector.java:113 - Exiting due to error while processing commit log during initialization. org.apache.cassandra.io.FSWriteError: java.nio.file.AccessDeniedException: /var/lib/cassandra/commitlog/CommitLog-7-1594961253708.log at org.apache.cassandra.db.commitlog.CommitLogSegment.(CommitLogSegment.java:180) at org.apache.cassandra.db.commitlog.MemoryMappedSegment.(MemoryMappedSegment.java:45) at org.apache.cassandra.db.commitlog.CommitLogSegment.createSegment(CommitLogSegment.java:137) at org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard.createSegment(CommitLogSegmentManagerStandard.java:66) at org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager$1.runMayThrow(AbstractCommitLogSegmentManager.java:114) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:748) Caused by: java.nio.file.AccessDeniedException: /var/lib/cassandra/commitlog/CommitLog-7-1594961253708.log at sun.nio.fs.UnixException.translateToIOException(UnixException.java:84) at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102) at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107) at sun.nio.fs.UnixFileSystemProvider.newFileChannel(UnixFileSystemProvider.java:177) at java.nio.channels.FileChannel.open(FileChannel.java:287) at java.nio.channels.FileChannel.open(FileChannel.java:335) at org.apache.cassandra.db.commitlog.CommitLogSegment.(CommitLogSegment.java:175) ... 7 common frames omitted -- 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-15954) When compaction gets interrupted, the exception should include the compactionId
David Capwell created CASSANDRA-15954: - Summary: When compaction gets interrupted, the exception should include the compactionId Key: CASSANDRA-15954 URL: https://issues.apache.org/jira/browse/CASSANDRA-15954 Project: Cassandra Issue Type: Improvement Components: Local/Compaction Reporter: David Capwell When we log in compaction we use the compactionId (or taskId), because of this we can figure out the start and end log lines for compaction. The issue is, when you interrupt a compaction task, we don’t log the compactionId, we instead log the tableId; for this reason it is much harder to attribute the interrupt to a single compactionId Examples {code} INFO 2020-07-15T18:04:51,670 [CompactionExecutor:144057] org.apache.cassandra.db.compaction.CompactionTask:158 - Compacting (5a463760-c700-11ea-8982-13c71a558319) [data/ks/table/ma-27-Data.db:level=0, data/ks/table/ ma-24-Data.db:level=0, ] INFO 2020-07-15T18:33:47,550 [CompactionExecutor:144057] org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor:1942 - Compaction interrupted: Compaction@057aa994-c35b-39ec-b74d-33fba2d13bbc(ks, table, 3105436904/8658590553)bytes {code} -- 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-15954) When compaction gets interrupted, the exception should include the compactionId
[ https://issues.apache.org/jira/browse/CASSANDRA-15954?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15954: -- Change Category: Operability Complexity: Low Hanging Fruit Status: Open (was: Triage Needed) > When compaction gets interrupted, the exception should include the > compactionId > --- > > Key: CASSANDRA-15954 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15954 > Project: Cassandra > Issue Type: Improvement > Components: Local/Compaction >Reporter: David Capwell >Priority: Normal > > When we log in compaction we use the compactionId (or taskId), because of > this we can figure out the start and end log lines for compaction. The issue > is, when you interrupt a compaction task, we don’t log the compactionId, we > instead log the tableId; for this reason it is much harder to attribute the > interrupt to a single compactionId > Examples > {code} > INFO 2020-07-15T18:04:51,670 [CompactionExecutor:144057] > org.apache.cassandra.db.compaction.CompactionTask:158 - Compacting > (5a463760-c700-11ea-8982-13c71a558319) [data/ks/table/ma-27-Data.db:level=0, > data/ks/table/ > ma-24-Data.db:level=0, ] > INFO 2020-07-15T18:33:47,550 [CompactionExecutor:144057] > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor:1942 - > Compaction interrupted: Compaction@057aa994-c35b-39ec-b74d-33fba2d13bbc(ks, > table, 3105436904/8658590553)bytes > {code} -- 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-15953) Support fetching all user tables to compare in Cassandra-diff
[ https://issues.apache.org/jira/browse/CASSANDRA-15953?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yifan Cai updated CASSANDRA-15953: -- Change Category: Operability Complexity: Normal Status: Open (was: Triage Needed) > Support fetching all user tables to compare in Cassandra-diff > - > > Key: CASSANDRA-15953 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15953 > Project: Cassandra > Issue Type: Improvement > Components: Tool/diff >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > > The spark diff job may fail to launch with kernel error "E2BIG: Argument list > too long", when passing a large list of keyspace table list to compare. > Proposing a mode to fetch all user tables from the clusters to be compared. > When the mode is on, the spark job ignores the parameter "keyspace_tables". -- 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-15953) Support fetching all user tables to compare in Cassandra-diff
Yifan Cai created CASSANDRA-15953: - Summary: Support fetching all user tables to compare in Cassandra-diff Key: CASSANDRA-15953 URL: https://issues.apache.org/jira/browse/CASSANDRA-15953 Project: Cassandra Issue Type: Improvement Components: Tool/diff Reporter: Yifan Cai Assignee: Yifan Cai The spark diff job may fail to launch with kernel error "E2BIG: Argument list too long", when passing a large list of keyspace table list to compare. Proposing a mode to fetch all user tables from the clusters to be compared. When the mode is on, the spark job ignores the parameter "keyspace_tables". -- 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-15945) Verify expected SSTable components are present at startup
[ https://issues.apache.org/jira/browse/CASSANDRA-15945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yifan Cai updated CASSANDRA-15945: -- Test and Documentation Plan: ci Status: Patch Available (was: Open) > Verify expected SSTable components are present at startup > - > > Key: CASSANDRA-15945 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15945 > Project: Cassandra > Issue Type: Bug > Components: Local/SSTable >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0 > > Time Spent: 10m > Remaining Estimate: 0h > > When Cassandra batch opens all sstables at startup, it should check that all > components listed in the TOC.txt are available. Compression is optional, so > there may not be a Compression.db component for all sstables, but if there > should be and it's missing we continue to load the sstable. The sstable is > unreadable without the compression metadata though, but we won't know this > until it's touched by a query or compaction. > To reliably use TOC.txt for verification, it needs CASSANDRA-10709 -- 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-15945) Verify expected SSTable components are present at startup
[ https://issues.apache.org/jira/browse/CASSANDRA-15945?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yifan Cai updated CASSANDRA-15945: -- Bug Category: Parent values: Correctness(12982)Level 1 values: Recoverable Corruption / Loss(12986) Complexity: Normal Discovered By: Adhoc Test Fix Version/s: 4.0 Severity: Low Status: Open (was: Triage Needed) > Verify expected SSTable components are present at startup > - > > Key: CASSANDRA-15945 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15945 > Project: Cassandra > Issue Type: Bug > Components: Local/SSTable >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0 > > Time Spent: 10m > Remaining Estimate: 0h > > When Cassandra batch opens all sstables at startup, it should check that all > components listed in the TOC.txt are available. Compression is optional, so > there may not be a Compression.db component for all sstables, but if there > should be and it's missing we continue to load the sstable. The sstable is > unreadable without the compression metadata though, but we won't know this > until it's touched by a query or compaction. > To reliably use TOC.txt for verification, it needs CASSANDRA-10709 -- 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-15945) Verify expected SSTable components are present at startup
[ https://issues.apache.org/jira/browse/CASSANDRA-15945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159542#comment-17159542 ] Yifan Cai commented on CASSANDRA-15945: --- PR: [https://github.com/apache/cassandra/pull/682] Code: [https://github.com/yifan-c/cassandra/tree/CASSANDRA-15945-verify-sstable-components] Test: [https://app.circleci.com/pipelines/github/yifan-c/cassandra] The patch provides the best-effort verification based on the entries in TOC file to verify the existence of the required {{CompressionInfo}} component, when opening a sstable. If the TOC file does not exist, the verification is a no-op. It only verifies the {{CompressionInfo}} component, because the other components are either mandatory and verified or we rebuild them on opening. Only the compressionInfo is mandatory when compression is enabled but no verification for it in the code base yet. See the table below. _Mandatory: must present on opening. Optional: can be absent on opening._ ||Component||Note|| |DATA|Mandatory. Verified.| |PRIMARY INDEX|Mandatory (if with validation). Verified.| |FILTER|Optional. Recreate if needed| |COMPRESSION INFO|Mandatory if compression enabled. | |STATS|Mandatory. Verified| |DIGEST|Optional. | |CRC|Optional.| |SUMMARY|Optional.| |TOC|Optional.| > Verify expected SSTable components are present at startup > - > > Key: CASSANDRA-15945 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15945 > Project: Cassandra > Issue Type: Bug > Components: Local/SSTable >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Time Spent: 10m > Remaining Estimate: 0h > > When Cassandra batch opens all sstables at startup, it should check that all > components listed in the TOC.txt are available. Compression is optional, so > there may not be a Compression.db component for all sstables, but if there > should be and it's missing we continue to load the sstable. The sstable is > unreadable without the compression metadata though, but we won't know this > until it's touched by a query or compaction. > To reliably use TOC.txt for verification, it needs CASSANDRA-10709 -- 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-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159525#comment-17159525 ] Caleb Rackliffe commented on CASSANDRA-15907: - [~adelapena] I tried to strip down the lazy rows approach a bit [here|https://github.com/maedhroz/cassandra/commit/c5abb49626da0141277de92e173fa8ed8062bcf3]. Now that I understand it a bit better, I'm a bit skeptical about whether we want to proceed. We already know that the partition-restricted case without a digest mismatch avoids all of this altogether. When there is a large number of non-conflicting rows at the start of the first-phase iterator, though, it seems like the price of avoiding row caching is creating a large number of {{CachedRowIterator}} objects. Maybe this is the right trade-off, but I'm not sure. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 4h 20m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtection}}, etc.) where we mention "replica-side > filtering protection" (which makes it seem like the coordinator doesn't > filter) rather than "replica filtering protection" (which sounds more like > what we actually do, which is protect ourselves against incorrect replica > filtering results). It's a minor fix, but would avoid confusion. > * The method call chain in {{DataResolver}} might be a bit simpler if we put > the {{repairedDataTracker}} in {{ResolveContext}}. > *Testing* > * I want to bite the bullet and get some basic tests for RFP (including any > guardrails we might add here) onto the in-JVM dtest framework. > *Guardrails* > * As it stands, we don't have a way to enforce an upper bound on the memory > usage of {{ReplicaFilteringProtection}} which caches row responses from the > first round of requests. (Remember, these are later used to merged with the > second round of results to complete the data for filtering.) Operators will > likely need a way to protect themselves, i.e. simply fail queries if they hit > a particular threshold rather than GC nodes into oblivion. (Having control > over limits and page sizes doesn't quite get us there, because stale results > _expand_ the number of incomplete results we must cache.) The fun question is > how we do this, with the primary axes being scope (per-query, global, etc.) > and granularity (per-partition, per-row, per-cell, actual heap usage, etc.). > My starting disposition on the right trade-off between > performance/complexity and accuracy is having something along the lines of > cached rows per query. Prior art suggests this probably makes sense alongside > things like {{tombstone_failure_threshold}} in {{cassandra.yaml}}. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: co
[jira] [Commented] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159482#comment-17159482 ] Caleb Rackliffe commented on CASSANDRA-15907: - bq. the main advantage of this approach is that in the absence of conflicts nothing needs to be cached Even though partition-restricted queries without digest mismatches will skip {{DataResolver}} entirely, this still helps in the case where we have a mismatch, but start with a large number of conflict-free rows, correct? If so, I think this is a benefit we can't ignore, given how likely we are to be dealing with partition-restricted queries. bq. Also we could also just let it unbounded to minimize the number of RFP queries The one thing that makes me a little uneasy about this is the extra logic we need to enforce the "target cache size". I propose we avoid that, simply leave the guardrails we've already got in place to avoid catastrophe (and excessive RFP queries), and see if that means we can simplify what remains (like having to clear the {{contents}} array list between batches). I'll try this and see how it looks and pull it into the main 3.0 branch if it works. Aside from that, I think the only remaining question would be verifying the safety of [aggressively clearing|https://github.com/apache/cassandra/pull/659/commits/30b8f4bebd95b3520b637d6d25d6bc16cb4d81a2] {{responses}}. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 4h 20m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtection}}, etc.) where we mention "replica-side > filtering protection" (which makes it seem like the coordinator doesn't > filter) rather than "replica filtering protection" (which sounds more like > what we actually do, which is protect ourselves against incorrect replica > filtering results). It's a minor fix, but would avoid confusion. > * The method call chain in {{DataResolver}} might be a bit simpler if we put > the {{repairedDataTracker}} in {{ResolveContext}}. > *Testing* > * I want to bite the bullet and get some basic tests for RFP (including any > guardrails we might add here) onto the in-JVM dtest framework. > *Guardrails* > * As it stands, we don't have a way to enforce an upper bound on the memory > usage of {{ReplicaFilteringProtection}} which caches row responses from the > first round of requests. (Remember, these are later used to merged with the > second round of results to complete the data for filtering.) Operators will > likely need a way to protect themselves, i.e. simply fail queries if they hit > a particular threshold rather than GC nodes into oblivion. (Having control > over limits and page sizes doesn't quite get us there, because stale results > _expand_ the number of incomplete results we must cache.) The fun question is > how we do this, w
[jira] [Commented] (CASSANDRA-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159450#comment-17159450 ] Brandon Williams commented on CASSANDRA-15952: -- 3.0: [https://ci-cassandra.apache.org/job/Cassandra-devbranch/212/] 3.11: https://ci-cassandra.apache.org/job/Cassandra-devbranch/213/ > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159440#comment-17159440 ] Ekaterina Dimitrova commented on CASSANDRA-15952: - [~brandon.williams], can you run CI for 3.0 and 3.11, please? If nothing pop up(it shouldn't), +1 from me. Thank you! > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ekaterina Dimitrova updated CASSANDRA-15952: Reviewers: Ekaterina Dimitrova, Ekaterina Dimitrova (was: Ekaterina Dimitrova) Ekaterina Dimitrova, Ekaterina Dimitrova (was: Ekaterina Dimitrova) Status: Review In Progress (was: Patch Available) > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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] [Comment Edited] (CASSANDRA-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159435#comment-17159435 ] Caleb Rackliffe edited comment on CASSANDRA-15952 at 7/16/20, 7:48 PM: --- dtest PR: https://github.com/apache/cassandra-dtest/pull/85 {{test_bootstrap_binary_disabled}} and {{test_resumable_bootstrap}} are both passing locally now consistently against 3.0 was (Author: maedhroz): dtest PR: https://github.com/apache/cassandra-dtest/pull/85 > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15952: Test and Documentation Plan: Verify that {{test_bootstrap_binary_disabled}} works against 3.0 and 3.11, and that the newly returned {{test_resumable_bootstrap}} works as well (was: Verify that test_bootstrap_binary_disabled works against 3.0 and 3.11, and that the newly returned {{test_resumable_bootstrap}} works as well) > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15952: Test and Documentation Plan: Verify that test_bootstrap_binary_disabled works against 3.0 and 3.11, and that the newly returned {{test_resumable_bootstrap}} works as well Status: Patch Available (was: In Progress) dtest PR: https://github.com/apache/cassandra-dtest/pull/85 > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159429#comment-17159429 ] Caleb Rackliffe commented on CASSANDRA-15952: - Fixing the test to check for the new log messages only in 4.0+ was simple enough. I've also taken the opportunity to add back {{test_resumable_bootstrap}}, which was apparently agreed upon [here|https://issues.apache.org/jira/browse/CASSANDRA-15667?focusedCommentId=17079380&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17079380] and just missed. > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15952: Reviewers: Ekaterina Dimitrova > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15693) Generating nodetool docs fails with python 3
[ https://issues.apache.org/jira/browse/CASSANDRA-15693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ekaterina Dimitrova updated CASSANDRA-15693: Reviewers: Ekaterina Dimitrova > Generating nodetool docs fails with python 3 > > > Key: CASSANDRA-15693 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15693 > Project: Cassandra > Issue Type: Bug > Components: Build >Reporter: Angelo Polo >Assignee: Angelo Polo >Priority: Normal > Labels: patch > Fix For: 4.0-beta > > Attachments: 0001-Fix-file-open-modes-for-python-3.patch > > > Building nodetool docs with the ant target 'gen-doc' fails for python 3. > Python 3 doesn't allow the file open mode "rw+". > {noformat} > gen-doc: > [exec] python convert_yaml_to_rst.py ../conf/cassandra.yaml > source/configuration/cassandra_config_file.rst > [exec] python gen-nodetool-docs.py > [exec] Makefile:64: recipe for target 'html' failed > [exec] Traceback (most recent call last): > [exec] File "gen-nodetool-docs.py", line 75, in > [exec] with open(helpfilename, "rw+") as helpfile: > [exec] ValueError: must have exactly one of create/read/write/append mode > [exec] make: *** [html] Error 1 > [exec] Result: 2 > {noformat} > Fails on and patch [^0001-Fix-file-open-modes-for-python-3.patch] tested on > the following platforms: Ubuntu 18.04.4 LTS with Python 3.6.9 and FreeBSD > 12.1-RELEASE-p1 with Python 3.7.7. -- 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-15693) Generating nodetool docs fails with python 3
[ https://issues.apache.org/jira/browse/CASSANDRA-15693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ekaterina Dimitrova updated CASSANDRA-15693: Test and Documentation Plan: Building nodetool docs with the ant target 'gen-doc' fails for python 3. Python 3 doesn't allow the file open mode "rw+". {noformat} gen-doc: [exec] python convert_yaml_to_rst.py ../conf/cassandra.yaml source/configuration/cassandra_config_file.rst [exec] python gen-nodetool-docs.py [exec] Makefile:64: recipe for target 'html' failed [exec] Traceback (most recent call last): [exec] File "gen-nodetool-docs.py", line 75, in [exec] with open(helpfilename, "rw+") as helpfile: [exec] ValueError: must have exactly one of create/read/write/append mode [exec] make: *** [html] Error 1 [exec] Result: 2 {noformat} Fails on and patch [^0001-Fix-file-open-modes-for-python-3.patch] tested on the following platforms: Ubuntu 18.04.4 LTS with Python 3.6.9 and FreeBSD 12.1-RELEASE-p1 with Python 3.7.7. Status: Patch Available (was: In Progress) > Generating nodetool docs fails with python 3 > > > Key: CASSANDRA-15693 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15693 > Project: Cassandra > Issue Type: Bug > Components: Build >Reporter: Angelo Polo >Assignee: Angelo Polo >Priority: Normal > Labels: patch > Attachments: 0001-Fix-file-open-modes-for-python-3.patch > > > Building nodetool docs with the ant target 'gen-doc' fails for python 3. > Python 3 doesn't allow the file open mode "rw+". > {noformat} > gen-doc: > [exec] python convert_yaml_to_rst.py ../conf/cassandra.yaml > source/configuration/cassandra_config_file.rst > [exec] python gen-nodetool-docs.py > [exec] Makefile:64: recipe for target 'html' failed > [exec] Traceback (most recent call last): > [exec] File "gen-nodetool-docs.py", line 75, in > [exec] with open(helpfilename, "rw+") as helpfile: > [exec] ValueError: must have exactly one of create/read/write/append mode > [exec] make: *** [html] Error 1 > [exec] Result: 2 > {noformat} > Fails on and patch [^0001-Fix-file-open-modes-for-python-3.patch] tested on > the following platforms: Ubuntu 18.04.4 LTS with Python 3.6.9 and FreeBSD > 12.1-RELEASE-p1 with Python 3.7.7. -- 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-15693) Generating nodetool docs fails with python 3
[ https://issues.apache.org/jira/browse/CASSANDRA-15693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ekaterina Dimitrova updated CASSANDRA-15693: Fix Version/s: 4.0-beta > Generating nodetool docs fails with python 3 > > > Key: CASSANDRA-15693 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15693 > Project: Cassandra > Issue Type: Bug > Components: Build >Reporter: Angelo Polo >Assignee: Angelo Polo >Priority: Normal > Labels: patch > Fix For: 4.0-beta > > Attachments: 0001-Fix-file-open-modes-for-python-3.patch > > > Building nodetool docs with the ant target 'gen-doc' fails for python 3. > Python 3 doesn't allow the file open mode "rw+". > {noformat} > gen-doc: > [exec] python convert_yaml_to_rst.py ../conf/cassandra.yaml > source/configuration/cassandra_config_file.rst > [exec] python gen-nodetool-docs.py > [exec] Makefile:64: recipe for target 'html' failed > [exec] Traceback (most recent call last): > [exec] File "gen-nodetool-docs.py", line 75, in > [exec] with open(helpfilename, "rw+") as helpfile: > [exec] ValueError: must have exactly one of create/read/write/append mode > [exec] make: *** [html] Error 1 > [exec] Result: 2 > {noformat} > Fails on and patch [^0001-Fix-file-open-modes-for-python-3.patch] tested on > the following platforms: Ubuntu 18.04.4 LTS with Python 3.6.9 and FreeBSD > 12.1-RELEASE-p1 with Python 3.7.7. -- 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-15693) Generating nodetool docs fails with python 3
[ https://issues.apache.org/jira/browse/CASSANDRA-15693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159407#comment-17159407 ] Ekaterina Dimitrova commented on CASSANDRA-15693: - Thank you. I will review it tomorrow. If you hit any other issues on FreeBSD, feel free to attach sub-tickets to CASSANDRA-15586. > Generating nodetool docs fails with python 3 > > > Key: CASSANDRA-15693 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15693 > Project: Cassandra > Issue Type: Bug > Components: Build >Reporter: Angelo Polo >Assignee: Angelo Polo >Priority: Normal > Labels: patch > Attachments: 0001-Fix-file-open-modes-for-python-3.patch > > > Building nodetool docs with the ant target 'gen-doc' fails for python 3. > Python 3 doesn't allow the file open mode "rw+". > {noformat} > gen-doc: > [exec] python convert_yaml_to_rst.py ../conf/cassandra.yaml > source/configuration/cassandra_config_file.rst > [exec] python gen-nodetool-docs.py > [exec] Makefile:64: recipe for target 'html' failed > [exec] Traceback (most recent call last): > [exec] File "gen-nodetool-docs.py", line 75, in > [exec] with open(helpfilename, "rw+") as helpfile: > [exec] ValueError: must have exactly one of create/read/write/append mode > [exec] make: *** [html] Error 1 > [exec] Result: 2 > {noformat} > Fails on and patch [^0001-Fix-file-open-modes-for-python-3.patch] tested on > the following platforms: Ubuntu 18.04.4 LTS with Python 3.6.9 and FreeBSD > 12.1-RELEASE-p1 with Python 3.7.7. -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
[ https://issues.apache.org/jira/browse/CASSANDRA-15952?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15952: Bug Category: Parent values: Correctness(12982)Level 1 values: Test Failure(12990) Complexity: Low Hanging Fruit Discovered By: Adhoc Test Fix Version/s: 3.11.x 3.0.x Severity: Low Status: Open (was: Triage Needed) > TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks > non-existent log messages for 3.0 and 3.11 > --- > > Key: CASSANDRA-15952 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-14525 added a bit of logging only in trunk that > {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes > is present in all versions. This should be as simple as making sure we only > assert around that on 4.0+. > See > https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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-15952) TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11
Caleb Rackliffe created CASSANDRA-15952: --- Summary: TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap checks non-existent log messages for 3.0 and 3.11 Key: CASSANDRA-15952 URL: https://issues.apache.org/jira/browse/CASSANDRA-15952 Project: Cassandra Issue Type: Bug Components: Test/dtest Reporter: Caleb Rackliffe Assignee: Caleb Rackliffe CASSANDRA-14525 added a bit of logging only in trunk that {{TestBootstrap::test_bootstrap_binary_disabled_resumable_bootstrap}} assumes is present in all versions. This should be as simple as making sure we only assert around that on 4.0+. See https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e/jobs/312 -- 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] [Comment Edited] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159370#comment-17159370 ] Andres de la Peña edited comment on CASSANDRA-15907 at 7/16/20, 5:50 PM: - [~maedhroz] I couldn't resist giving a try to the per-row lazy pre-fetch, I have left it [here|https://github.com/adelapena/cassandra/commit/accf2a47c341875942b0d8b06c016cc0d66d62cb]. Instead of consuming all the contents of each merged partition, it consumes them row-per-row until there are replica contents. That way, if there are no conflicts, it only caches one row per replica, instead of an entire partition per replica. Also, iif it finds that there are rows to fetch from the replica, it advances the first phase merged row iterator a bit more until reaching a certain cache size, trying to find a balance between the cache size and the number of RFP queries. Right now that desired cache size is hardcoded to 100, but we could use a config property, or the query limit, for example. Also we could also just let it unbounded to minimize the number of RFP queries, the main advantage of this approach is that in the absence of conflicts nothing needs to be cached. That should benefit the most common case, which is when there are no conflicts. To illustrate how the per-row approach behaves, let's see this example: {code:python} self._prepare_cluster( create_table="CREATE TABLE t (k int, c int, v text, PRIMARY KEY(k, c))", create_index="CREATE INDEX ON t(v)", both_nodes=["INSERT INTO t (k, c, v) VALUES (0, 0, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 1, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 2, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 3, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 4, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 5, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 6, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 7, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 8, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 9, 'old')"], only_node1=["INSERT INTO t (k, c, v) VALUES (0, 4, 'new')", "INSERT INTO t (k, c, v) VALUES (0, 6, 'new')"]) self._assert_all("SELECT c FROM t WHERE v = 'old'", rows=[[0], [1], [2], [3], [5], [7], [8], [9]]) {code} Without the per-row approach we cached 20 rows (10 per replica) and issued one single RFP query. In contrast, with the per-row approach the behaviour should be: * If the target cache size is very high or unbounded, we will cache a max of 12 rows and we will need 1 RFP queries. There are less cached row because we don't cache rows until the first conflict is found in the fourth row. * If the target cache size is 2, we will cache a max of 8 rows and we will need 1 RFP queries. This is because the two conflicts fit into the same window of cached rows, and once they are fetched we don't need to cache more rows. * If we use a target cache size of 1, we will cache a max of 6 rows but, differently from before, we will need 2 separate RFP queries. * If there are no conflicts we will only have one cached row per replica; the current one. Note that consuming rows from the first phase iterator to populate the cache can still produce an unlimited growth of the cache, so we still need the guardrail. The configurable target cache size that I mention is only used to try to find a balance between cache size and grouping of primary keys to fetch. was (Author: adelapena): [~maedhroz] I couldn't resist giving a try to the making the per-row lazy pre-fetch, I have left it [here|https://github.com/adelapena/cassandra/commit/accf2a47c341875942b0d8b06c016cc0d66d62cb]. Instead of consuming all the contents of each merged partition, it consumes them row-per-row until there are replica contents. That way, if there are no conflicts, it only caches one row per replica, instead of an entire partition per replica. Also, iif it finds that there are rows to fetch from the replica, it advances the first phase merged row iterator a bit more until reaching a certain cache size, trying to find a balance between the cache size and the number of RFP queries. Right now that desired cache size is hardcoded to 100, but we could use a config property, or the query limit, for example. Also we could also just let it unbounded to minimize the number of RFP queries, the main advantage of this approach is that in the absence of conflicts nothing needs to be cached. That should benefit the most common case, which is when there are no conflicts. To illustrate how the per-row approach behaves, let's see this example: {code:python} self._prepare_cluster( create_table="CREATE TABLE t (k int, c int, v text, PRIMARY KEY(k, c))", create_index="CREATE INDEX ON t(v)",
[jira] [Commented] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159370#comment-17159370 ] Andres de la Peña commented on CASSANDRA-15907: --- [~maedhroz] I couldn't resist giving a try to the making the per-row lazy pre-fetch, I have left it [here|https://github.com/adelapena/cassandra/commit/accf2a47c341875942b0d8b06c016cc0d66d62cb]. Instead of consuming all the contents of each merged partition, it consumes them row-per-row until there are replica contents. That way, if there are no conflicts, it only caches one row per replica, instead of an entire partition per replica. Also, iif it finds that there are rows to fetch from the replica, it advances the first phase merged row iterator a bit more until reaching a certain cache size, trying to find a balance between the cache size and the number of RFP queries. Right now that desired cache size is hardcoded to 100, but we could use a config property, or the query limit, for example. Also we could also just let it unbounded to minimize the number of RFP queries, the main advantage of this approach is that in the absence of conflicts nothing needs to be cached. That should benefit the most common case, which is when there are no conflicts. To illustrate how the per-row approach behaves, let's see this example: {code:python} self._prepare_cluster( create_table="CREATE TABLE t (k int, c int, v text, PRIMARY KEY(k, c))", create_index="CREATE INDEX ON t(v)", both_nodes=["INSERT INTO t (k, c, v) VALUES (0, 0, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 1, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 2, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 3, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 4, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 5, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 6, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 7, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 8, 'old')", "INSERT INTO t (k, c, v) VALUES (0, 9, 'old')"], only_node1=["INSERT INTO t (k, c, v) VALUES (0, 4, 'new')", "INSERT INTO t (k, c, v) VALUES (0, 6, 'new')"]) self._assert_all("SELECT c FROM t WHERE v = 'old'", rows=[[0], [1], [2], [3], [5], [7], [8], [9]]) {code} Without the per-row approach we cached 20 rows (10 per replica) and issued one single RFP query. In contrast, with the per-row approach: * If the target cache size is very high or unbounded, we will cache a max of 12 rows and we will need 1 RFP queries. * If the target cache size is 2, we will cache a max of 8 rows and we will need 1 RFP queries. * If we use a target cache size of 1, we will cache a max of 6 rows but, differently from before, we will need 2 separate RFP queries. * If there are no conflicts we will only have one cached row per replica; the current one. Note that consuming rows from the fist phase iterator to populate the cache can still produce an unlimited growth of the cache, so we still need the guardrail. The (perhaps) configurable target cache size that I mention is only used to try to find a balance between cache size and grouping of primary keys to fetch. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 4h 20m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method
[jira] [Comment Edited] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17152248#comment-17152248 ] Caleb Rackliffe edited comment on CASSANDRA-15907 at 7/16/20, 4:19 PM: --- [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/57/workflows/b408ce8a-ee68-47bf-b9f1-eb9541e9827e] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. was (Author: maedhroz): [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/51/workflows/4fd639f6-6523-4520-961e-5b5c384a13b3] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 4h 10m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtection}}, etc.) where we mention "replica-side > filtering protection" (which makes it seem like the coordinator doesn't > filter) rather than "replica filtering protection" (which sounds more like > what we actually do, which
[jira] [Updated] (CASSANDRA-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-15191: -- Fix Version/s: 4.0-beta1 3.11.7 > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 3.11.7, 4.0-beta1 > > Attachments: log.txt > > Time Spent: 10m > Remaining Estimate: 0h > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > The core reason for ignoring that exception is that thrown exception in > AbstractLocalAwareExecturorService is not CorruptSSTableException but it is > RuntimeException and that exception is as its cause. Hence it is better if we > handle this in JVMStabilityInspector which can recursively examine it, hence > act accordingly. > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- 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-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159292#comment-17159292 ] Stefan Miklosovic commented on CASSANDRA-15191: --- Hi [~jeromatron] and [~Bereng], could you review this, please? I ll create patch for trunk if proposed solution is fine here. > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Attachments: log.txt > > Time Spent: 10m > Remaining Estimate: 0h > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > The core reason for ignoring that exception is that thrown exception in > AbstractLocalAwareExecturorService is not CorruptSSTableException but it is > RuntimeException and that exception is as its cause. Hence it is better if we > handle this in JVMStabilityInspector which can recursively examine it, hence > act accordingly. > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- 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-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-15191: -- Description: There is a bug when disk_failure_policy is set to stop_paranoid and CorruptSSTableException is thrown after server is up. The problem is that this setting is ignored. Normally, it should stop gossip and transport but it just continues to serve requests and an exception is just logged. This patch unifies the exception handling in JVMStabilityInspector and code is reworked in such way that this inspector acts as a central place where such exceptions are inspected. The core reason for ignoring that exception is that thrown exception in AbstractLocalAwareExecturorService is not CorruptSSTableException but it is RuntimeException and that exception is as its cause. Hence it is better if we handle this in JVMStabilityInspector which can recursively examine it, hence act accordingly. Behaviour before: stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException is thrown, e.g. on a regular select statement Behaviour after: Gossip and transport (cql) is turned off, JVM is still up for further investigation e.g. by jmx. was: There is a bug when disk_failure_policy is set to stop_paranoid and CorruptSSTableException is thrown after server is up. The problem is that this setting is ignored. Normally, it should stop gossip and transport but it just continues to serve requests and an exception is just logged. This patch unifies the exception handling in JVMStabilityInspector and code is reworked in such way that this inspector acts as a central place where such exceptions are inspected. Behaviour before: stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException is thrown, e.g. on a regular select statement Behaviour after: Gossip and transport (cql) is turned off, JVM is still up for further investigation e.g. by jmx. > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Attachments: log.txt > > Time Spent: 10m > Remaining Estimate: 0h > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > The core reason for ignoring that exception is that thrown exception in > AbstractLocalAwareExecturorService is not CorruptSSTableException but it is > RuntimeException and that exception is as its cause. Hence it is better if we > handle this in JVMStabilityInspector which can recursively examine it, hence > act accordingly. > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- 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-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159287#comment-17159287 ] Stefan Miklosovic commented on CASSANDRA-15191: --- PR for 3.11 [https://github.com/apache/cassandra/pull/681] > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Attachments: log.txt > > Time Spent: 10m > Remaining Estimate: 0h > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- 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-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-15191: -- Bug Category: Parent values: Correctness(12982) Complexity: Normal Component/s: Local/Config Severity: Normal Assignee: Stefan Miklosovic Status: Open (was: Triage Needed) > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Attachments: log.txt > > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- 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-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-15191: -- Description: There is a bug when disk_failure_policy is set to stop_paranoid and CorruptSSTableException is thrown after server is up. The problem is that this setting is ignored. Normally, it should stop gossip and transport but it just continues to serve requests and an exception is just logged. This patch unifies the exception handling in JVMStabilityInspector and code is reworked in such way that this inspector acts as a central place where such exceptions are inspected. Behaviour before: stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException is thrown, e.g. on a regular select statement Behaviour after: Gossip and transport (cql) is turned off, JVM is still up for further investigation e.g. by jmx. was: If the {{stop_paranoid}} disk failure policy is triggered during startup (for example by a compaction) before the node reaches CassandraDaemon.start(), CassandraDaemon.start() will start the client transport services and start listening for client connections regardless of the previous corrupt sstable exception. > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug >Reporter: Vincent White >Priority: Normal > Attachments: log.txt > > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- 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-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-15191: -- Summary: stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up (was: stop_paranoid disk failure policy race condition on startup) > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug >Reporter: Vincent White >Priority: Normal > Attachments: log.txt > > > If the {{stop_paranoid}} disk failure policy is triggered during startup (for > example by a compaction) before the node reaches CassandraDaemon.start(), > CassandraDaemon.start() will start the client transport services and start > listening for client connections regardless of the previous corrupt sstable > exception. > > -- 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-15406) Add command to show the progress of data streaming and index build
[ https://issues.apache.org/jira/browse/CASSANDRA-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159238#comment-17159238 ] Berenguer Blasi commented on CASSANDRA-15406: - Triggered Ci run: https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/211/pipeline > Add command to show the progress of data streaming and index build > --- > > Key: CASSANDRA-15406 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15406 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Streaming, Legacy/Streaming and Messaging, > Tool/nodetool >Reporter: maxwellguo >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 4.0, 4.x > > Time Spent: 10m > Remaining Estimate: 0h > > I found that we should supply a command to show the progress of streaming > when we do the operation of bootstrap/move/decommission/removenode. For when > do data streaming , noboday knows which steps there program are in , so I > think a command to show the joing/leaving node's is needed . > > PR [https://github.com/apache/cassandra/pull/558] -- 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-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159194#comment-17159194 ] Alex Petrov commented on CASSANDRA-15299: - I think that renaming is justified in this case. I've been looking at this patch for some time, and I still have to make a mental effort to translate "Frame" into "Envelope" and so on. I don't think merge size of the patch is a concern here, neither is backport size. We have to be forward-looking, and most often people develop patches against trunk anyways. I'd rather make a mental switch every time I'm looking at the old code, since this will allow for a simpler, more consistent naming in the code that's most actively worked on. > CASSANDRA-13304 follow-up: improve checksumming and compression in protocol > v5-beta > --- > > Key: CASSANDRA-15299 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15299 > Project: Cassandra > Issue Type: Improvement > Components: Messaging/Client >Reporter: Aleksey Yeschenko >Assignee: Alex Petrov >Priority: Normal > Labels: protocolv5 > Fix For: 4.0-alpha > > > CASSANDRA-13304 made an important improvement to our native protocol: it > introduced checksumming/CRC32 to request and response bodies. It’s an > important step forward, but it doesn’t cover the entire stream. In > particular, the message header is not covered by a checksum or a crc, which > poses a correctness issue if, for example, {{streamId}} gets corrupted. > Additionally, we aren’t quite using CRC32 correctly, in two ways: > 1. We are calculating the CRC32 of the *decompressed* value instead of > computing the CRC32 on the bytes written on the wire - losing the properties > of the CRC32. In some cases, due to this sequencing, attempting to decompress > a corrupt stream can cause a segfault by LZ4. > 2. When using CRC32, the CRC32 value is written in the incorrect byte order, > also losing some of the protections. > See https://users.ece.cmu.edu/~koopman/pubs/KoopmanCRCWebinar9May2012.pdf for > explanation for the two points above. > Separately, there are some long-standing issues with the protocol - since > *way* before CASSANDRA-13304. Importantly, both checksumming and compression > operate on individual message bodies rather than frames of multiple complete > messages. In reality, this has several important additional downsides. To > name a couple: > # For compression, we are getting poor compression ratios for smaller > messages - when operating on tiny sequences of bytes. In reality, for most > small requests and responses we are discarding the compressed value as it’d > be smaller than the uncompressed one - incurring both redundant allocations > and compressions. > # For checksumming and CRC32 we pay a high overhead price for small messages. > 4 bytes extra is *a lot* for an empty write response, for example. > To address the correctness issue of {{streamId}} not being covered by the > checksum/CRC32 and the inefficiency in compression and checksumming/CRC32, we > should switch to a framing protocol with multiple messages in a single frame. > I suggest we reuse the framing protocol recently implemented for internode > messaging in CASSANDRA-15066 to the extent that its logic can be borrowed, > and that we do it before native protocol v5 graduates from beta. See > https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/FrameDecoderCrc.java > and > https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/FrameDecoderLZ4.java. -- 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-15942) Tooling testing foundation
[ https://issues.apache.org/jira/browse/CASSANDRA-15942?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-15942: - Reviewers: Robert Stupp, Robert Stupp (was: Robert Stupp) Robert Stupp, Robert Stupp Status: Review In Progress (was: Patch Available) Left a comment on the PR. Otherwise the patch looks good (and familiar ;) ). Ran a few of the changed tests and those passed. +1 w/ the PR comment (assume CI won't complain) > Tooling testing foundation > -- > > Key: CASSANDRA-15942 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15942 > Project: Cassandra > Issue Type: Improvement > Components: Test/unit >Reporter: Berenguer Blasi >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Tooling testing is not as thorough as we'd like it to be. The purpose of this > ticket, under the umbrella of CASSANDRA-15583, is to provide the foundations > to make it easy for devs, migrate the few existing tests as a POC and put > that forward as a proposal to build on top of. -- 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-15942) Tooling testing foundation
[ https://issues.apache.org/jira/browse/CASSANDRA-15942?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-15942: - Status: Ready to Commit (was: Review In Progress) > Tooling testing foundation > -- > > Key: CASSANDRA-15942 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15942 > Project: Cassandra > Issue Type: Improvement > Components: Test/unit >Reporter: Berenguer Blasi >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Tooling testing is not as thorough as we'd like it to be. The purpose of this > ticket, under the umbrella of CASSANDRA-15583, is to provide the foundations > to make it easy for devs, migrate the few existing tests as a POC and put > that forward as a proposal to build on top of. -- 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-15814) order by descending on frozen list not working
[ https://issues.apache.org/jira/browse/CASSANDRA-15814?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Berenguer Blasi updated CASSANDRA-15814: Reviewers: Berenguer Blasi > order by descending on frozen list not working > -- > > Key: CASSANDRA-15814 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15814 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter >Reporter: Felipe Perez >Assignee: Andres de la Peña >Priority: Normal > Time Spent: 50m > Remaining Estimate: 0h > > By creating a table like the following: > {code:java} > CREATE TABLE IF NOT EXISTS software ( > name ascii, > version frozen>, > data ascii, > PRIMARY KEY(name,version) > ) > {code} > It works and version is ordered in an ascending order. But when trying to > order in descending order: > {code:java} > CREATE TABLE IF NOT EXISTS software ( > name ascii, > version frozen>, > data ascii, > PRIMARY KEY(name,version) > ) WITH CLUSTERING ORDER BY (version DESC); > {code} > The table is created normally, but when trying to insert a row: > {code:java} > insert into software(name, version) values ('t1', [2,10,30,40,50]); > {code} > Cassandra throws an error: > {code:java} > InvalidRequest: Error from server: code=2200 [Invalid query] message="Invalid > list literal for version of type frozen>" > {code} > The goal here is that I would like to get the last version of a software. > -- 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-15951) Invalid version value: 4.0~alpha4
[ https://issues.apache.org/jira/browse/CASSANDRA-15951?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Erick Ramirez updated CASSANDRA-15951: -- Resolution: Duplicate Status: Resolved (was: Open) > Invalid version value: 4.0~alpha4 > - > > Key: CASSANDRA-15951 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15951 > Project: Cassandra > Issue Type: Bug > Components: Local/Startup and Shutdown >Reporter: rafi saeputra >Assignee: Erick Ramirez >Priority: Normal > Fix For: 4.0-beta1 > > > hello : > i was started cassandra 4.0 in Centos 7. When i started everything is fine, > but when i stop then start again i get an error like this. > ERROR [main] 2020-07-16 14:25:11,330 CassandraDaemon.java:789 - Exception > encountered during startup > java.lang.IllegalArgumentException: Invalid version value: 4.0~alpha4 > at > org.apache.cassandra.utils.CassandraVersion.(CassandraVersion.java:64) > at > org.apache.cassandra.io.sstable.SSTableHeaderFix.fixNonFrozenUDTIfUpgradeFrom30(SSTableHeaderFix.java:84) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:250) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:650) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:767) -- 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-15951) Invalid version value: 4.0~alpha4
[ https://issues.apache.org/jira/browse/CASSANDRA-15951?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Erick Ramirez updated CASSANDRA-15951: -- Bug Category: Parent values: Code(13163) Complexity: Normal Discovered By: User Report Fix Version/s: 4.0-beta1 Severity: Normal Assignee: Erick Ramirez Status: Open (was: Triage Needed) Marking as duplicate of CASSANDRA-15830. > Invalid version value: 4.0~alpha4 > - > > Key: CASSANDRA-15951 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15951 > Project: Cassandra > Issue Type: Bug > Components: Local/Startup and Shutdown >Reporter: rafi saeputra >Assignee: Erick Ramirez >Priority: Normal > Fix For: 4.0-beta1 > > > hello : > i was started cassandra 4.0 in Centos 7. When i started everything is fine, > but when i stop then start again i get an error like this. > ERROR [main] 2020-07-16 14:25:11,330 CassandraDaemon.java:789 - Exception > encountered during startup > java.lang.IllegalArgumentException: Invalid version value: 4.0~alpha4 > at > org.apache.cassandra.utils.CassandraVersion.(CassandraVersion.java:64) > at > org.apache.cassandra.io.sstable.SSTableHeaderFix.fixNonFrozenUDTIfUpgradeFrom30(SSTableHeaderFix.java:84) > at > org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:250) > at > org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:650) > at > org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:767) -- 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-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159037#comment-17159037 ] Berenguer Blasi commented on CASSANDRA-15792: - Anything else you need from me here [~gianluca]? > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- 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-15406) Add command to show the progress of data streaming and index build
[ https://issues.apache.org/jira/browse/CASSANDRA-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17159016#comment-17159016 ] Stefan Miklosovic commented on CASSANDRA-15406: --- [~Bereng] [~blerer] done [https://github.com/apache/cassandra/pull/558] > Add command to show the progress of data streaming and index build > --- > > Key: CASSANDRA-15406 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15406 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Streaming, Legacy/Streaming and Messaging, > Tool/nodetool >Reporter: maxwellguo >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 4.0, 4.x > > Time Spent: 10m > Remaining Estimate: 0h > > I found that we should supply a command to show the progress of streaming > when we do the operation of bootstrap/move/decommission/removenode. For when > do data streaming , noboday knows which steps there program are in , so I > think a command to show the joing/leaving node's is needed . > > PR [https://github.com/apache/cassandra/pull/558] -- 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-15583) 4.0 quality testing: Tooling, Bundled and First Party
[ https://issues.apache.org/jira/browse/CASSANDRA-15583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17158983#comment-17158983 ] Berenguer Blasi commented on CASSANDRA-15583: - CASSANDRA-15942 is ready for a cursory review to +1 the POC and approach to be used from now on. > 4.0 quality testing: Tooling, Bundled and First Party > - > > Key: CASSANDRA-15583 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15583 > Project: Cassandra > Issue Type: Task > Components: Test/dtest, Test/unit >Reporter: Josh McKenzie >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Reference [doc from > NGCC|https://docs.google.com/document/d/1uhUOp7wpE9ZXNDgxoCZHejHt5SO4Qw1dArZqqsJccyQ/edit#] > for context. > *Shepherd: Sam Tunnicliffe* > Test plans should cover bundled first-party tooling and CLIs such as > nodetool, cqlsh, and new tools supporting full query and audit logging > (CASSANDRA-13983, CASSANDRA-12151). -- 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-15857) Frozen RawTuple is not annotated with frozen in the toString method
[ https://issues.apache.org/jira/browse/CASSANDRA-15857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17158981#comment-17158981 ] Yifan Cai commented on CASSANDRA-15857: --- I can confirm that in 4.0, non-frozen UDT cannot be ever dropped. So it is safe to freeze a tuple unconditionally. {code:java} [cqlsh 5.0.1 | Cassandra 4.0-beta1-SNAPSHOT | CQL spec 3.4.5 | Native protocol v4] cqlsh> CREATE TYPE ks.some_type ( i int, t text ); cqlsh> CREATE TABLE ks.t ( pk int primary key, t some_type); cqlsh> ALTER TABLE ks.t DROP t; InvalidRequest: Error from server: code=2200 [Invalid query] message="Cannot drop non-frozen column t of user type some_type" {code} An optional [commit (freeze tuples unconditionally)|https://github.com/apache/cassandra/pull/669/commits/5e7577141de4ebe2c5c284fd7c22d0dd8f49aea0] was added to the patch. We can totally drop the commit if not like it. All raw tuples are now always frozen. Because of that, rejections on frozen raw tuple in CreateFunctionStatement and CreateAggregateStatement are removed. The rejections were added mainly to make the aggregate and function flexible to both frozen and non-frozen types, according to comments in CASSANDRA-9186. It prevents adding frozen in the declaration of aggregate and function, in order to not confuse users. Given that tuples are always frozen, user should be clear that 'tuple' and 'frozen tuple' produce the same effect. Therefore, it could be confusion-free to allow both form for tuple in the create statements for aggregates and functions. > Frozen RawTuple is not annotated with frozen in the toString method > --- > > Key: CASSANDRA-15857 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15857 > Project: Cassandra > Issue Type: Bug > Components: Legacy/CQL >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0, 3.11.x > > Time Spent: 10m > Remaining Estimate: 0h > > All raw types (e.g. RawCollection, RawUT) that supports freezing wraps the > type name with 'frozen<>' in the toString method, except RawTuple. > Therefore, the RawTuple::toString output misses the frozen wrapper. > Tuple is always frozen. However since CASSANDRA-15035, it throws when the > inner tuple is not explicitly wrapped with frozen within a collection. > The method, CQL3Type.Raw::toString, is referenced at multiple places in the > source. For example, referenced in CreateTypeStatement.Raw and involved in > CQLSSTableWriter. Another example is that it is called to produce the > SchemaChange at several AlterSchemaStatement implementations. > A test can prove that missing the frozen wrapper causes exception when > building CQLSSTableWriter for user types defined like below. Note that the > inner tuple is wrapped with frozen in the initial CQL statement. > {code:java} > CREATE TYPE ks.fooType ( f list>> ) > {code} > {code:java} > org.apache.cassandra.exceptions.InvalidRequestException: Non-frozen tuples > are not allowed inside collections: list> > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.throwNestedNonFrozenError(CQL3Type.java:710) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepare(CQL3Type.java:669) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepareInternal(CQL3Type.java:661) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.lambda$prepare$1(Types.java:341) > at > java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) > at > java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) > at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) > at > java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) > at > java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) > at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) > at > java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.prepare(Types.java:342) > at org.apache.cassandra.schema.Types$RawBuilder.build(Types.java:291) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.createTypes(CQLSSTableWriter.java:551) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.build(CQLSSTableWriter.java:527) > {code} -- 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-15406) Add command to show the progress of data streaming and index build
[ https://issues.apache.org/jira/browse/CASSANDRA-15406?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17158980#comment-17158980 ] Berenguer Blasi commented on CASSANDRA-15406: - [~stefan.miklosovic] I would rebase once again, yeah I'm OCD about it apologies but it will take you just 1m :-) As Benjamin is going to be on holidays for a few weeks I'd suggest you jump on Slack and ask for sbdy to run CI for you? > Add command to show the progress of data streaming and index build > --- > > Key: CASSANDRA-15406 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15406 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Streaming, Legacy/Streaming and Messaging, > Tool/nodetool >Reporter: maxwellguo >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 4.0, 4.x > > Time Spent: 10m > Remaining Estimate: 0h > > I found that we should supply a command to show the progress of streaming > when we do the operation of bootstrap/move/decommission/removenode. For when > do data streaming , noboday knows which steps there program are in , so I > think a command to show the joing/leaving node's is needed . > > PR [https://github.com/apache/cassandra/pull/558] -- 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-15951) Invalid version value: 4.0~alpha4
rafi saeputra created CASSANDRA-15951: - Summary: Invalid version value: 4.0~alpha4 Key: CASSANDRA-15951 URL: https://issues.apache.org/jira/browse/CASSANDRA-15951 Project: Cassandra Issue Type: Bug Components: Local/Startup and Shutdown Reporter: rafi saeputra hello : i was started cassandra 4.0 in Centos 7. When i started everything is fine, but when i stop then start again i get an error like this. ERROR [main] 2020-07-16 14:25:11,330 CassandraDaemon.java:789 - Exception encountered during startup java.lang.IllegalArgumentException: Invalid version value: 4.0~alpha4 at org.apache.cassandra.utils.CassandraVersion.(CassandraVersion.java:64) at org.apache.cassandra.io.sstable.SSTableHeaderFix.fixNonFrozenUDTIfUpgradeFrom30(SSTableHeaderFix.java:84) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:250) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:650) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:767) -- 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