[jira] [Commented] (CASSANDRA-17116) When zero-copy-streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-17116:
---

patch breaks streaming in python dtest; nice!

> When zero-copy-streaming sees a channel close this triggers the disk failure 
> policy
> ---
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17116) When zero-copy-streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-17116:
---

To show that the test actually detects the issue, remove the core of the patch 
which fixes the issue

{code}
git checkout trunk src/java/org/apache/cassandra/db/streaming 
test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java
 src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java
{code}

> When zero-copy-streaming sees a channel close this triggers the disk failure 
> policy
> ---
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17118:
--
  Fix Version/s: (was: 4.x)
 NA
  Since Version: NA
Source Control Link: 
https://github.com/apache/cassandra/commit/5b4d3692664172546b25c765f89c94e61400d873
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: NA
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17118:
--
Status: Ready to Commit  (was: Review In Progress)

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: 4.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17118:
--
Reviewers: David Capwell, David Capwell  (was: David Capwell)
   David Capwell, David Capwell
   Status: Review In Progress  (was: Patch Available)

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: 4.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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



[cassandra] branch trunk updated: coordinator_read_size had wrong name in cassandra.yaml

2021-11-03 Thread dcapwell
This is an automated email from the ASF dual-hosted git repository.

dcapwell pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5b4d369  coordinator_read_size had wrong name in cassandra.yaml
5b4d369 is described below

commit 5b4d3692664172546b25c765f89c94e61400d873
Author: dcapwell 
AuthorDate: Wed Nov 3 15:32:40 2021 -0700

coordinator_read_size had wrong name in cassandra.yaml

patch by Ekaterina Dimitrova, reviewed by David Capwell for CASSANDRA-17118
---
 conf/cassandra.yaml | 42 +-
 1 file changed, 21 insertions(+), 21 deletions(-)

diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 65eb385..92a91c6 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1543,24 +1543,24 @@ enable_drop_compact_storage: false
 
 # Enables tracking warnings/aborts across all replicas for reporting back to 
client.
 # See: CASSANDRA-16850
-#track_warnings:
-#  # Scheduled to enable in 4.2
-#  enabled: false
-#  # When track_warnings.enabled: true, this tracks the materialized size 
of a query on the
-#  # coordinator. If coordinator_large_read.warn_threshold_kb is greater 
than 0, this will emit a warning
-#  # to clients with details on what query triggered this as well as the 
size of the result set; if
-#  # coordinator_large_read.abort_threshold_kb is greater than 0, this 
will abort the query after it
-#  # has exceeded this threshold, returning a read error to the user.
-#  coordinator_large_read:
-#  warn_threshold_kb: 0
-#  abort_threshold_kb: 0
-#  # When track_warnings.enabled: true, this tracks the size of the local 
read (as defined by
-#  # heap size), and will warn/abort based off these thresholds; 0 
disables these checks.
-#  local_read_size:
-#  warn_threshold_kb: 0
-#  abort_threshold_kb: 0
-#  # When track_warnings.enabled: true, this tracks the expected memory 
size of the RowIndexEntry
-#  # and will warn/abort based off these thresholds; 0 disables these 
checks.
-#  row_index_size:
-#  warn_threshold_kb: 0
-#  abort_threshold_kb: 0
+# track_warnings:
+# Scheduled to enable in 4.2
+# enabled: true
+# When track_warnings.enabled: true, this tracks the materialized size of a 
query on the
+# coordinator. If coordinator_large_read.warn_threshold_kb is greater than 0, 
this will emit a warning
+# to clients with details on what query triggered this as well as the size of 
the result set; if
+# coordinator_large_read.abort_threshold_kb is greater than 0, this will abort 
the query after it
+# has exceeded this threshold, returning a read error to the user.
+# coordinator_read_size:
+# warn_threshold_kb: 0
+# abort_threshold_kb: 0
+# When track_warnings.enabled: true, this tracks the size of the local read 
(as defined by
+# heap size), and will warn/abort based off these thresholds; 0 disables these 
checks.
+# local_read_size:
+# warn_threshold_kb: 0
+# abort_threshold_kb: 0
+# When track_warnings.enabled: true, this tracks the expected memory size of 
the RowIndexEntry
+# and will warn/abort based off these thresholds; 0 disables these checks.
+# row_index_size:
+# warn_threshold_kb: 0
+# abort_threshold_kb: 0

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



[jira] [Commented] (CASSANDRA-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-17118:
---

PR is https://github.com/apache/cassandra/pull/1302

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: 4.x
>
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova updated CASSANDRA-17118:

Test and Documentation Plan: 
https://issues.apache.org/jira/browse/CASSANDRA-17118?focusedCommentId=17438319&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17438319
 Status: Patch Available  (was: In Progress)

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: 4.x
>
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova commented on CASSANDRA-17118:
-

I think this change requires only manual testing. CI cannot catch it as it uses 
the test/conf file instead where the parameter has already the correct name.

Patch posted 
[here|https://github.com/ekaterinadimitrova2/cassandra/pull/new/CASSANDRA-17118-trunk]

[~dcapwell], can you review, please?

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: 4.x
>
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova updated CASSANDRA-17118:

Fix Version/s: 4.x

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
> Fix For: 4.x
>
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread Ekaterina Dimitrova (Jira)
Ekaterina Dimitrova created CASSANDRA-17118:
---

 Summary: coordinator_large_read should be actually  
coordinator_read_size in cassandra.yaml
 Key: CASSANDRA-17118
 URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
 Project: Cassandra
  Issue Type: Bug
Reporter: Ekaterina Dimitrova


coordinator_large_read should be actually called coordinator_read_size in 
cassandra.yaml

After refactoring the name was changed in the test config but not in the 
default file.

The issue was not caught because track_warnings is currently disabled and 
commented out.



--
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-17118) coordinator_large_read should be actually coordinator_read_size in cassandra.yaml

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova updated CASSANDRA-17118:

 Bug Category: Parent values: Correctness(12982)
   Complexity: Low Hanging Fruit
  Component/s: Local/Config
Discovered By: User Report
 Severity: Low
 Assignee: Ekaterina Dimitrova
   Status: Open  (was: Triage Needed)

> coordinator_large_read should be actually  coordinator_read_size in 
> cassandra.yaml
> --
>
> Key: CASSANDRA-17118
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17118
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Config
>Reporter: Ekaterina Dimitrova
>Assignee: Ekaterina Dimitrova
>Priority: Normal
>
> coordinator_large_read should be actually called coordinator_read_size in 
> cassandra.yaml
> After refactoring the name was changed in the test config but not in the 
> default file.
> The issue was not caught because track_warnings is currently disabled and 
> commented out.



--
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-16957) Actively update auth caches in the background

2021-11-03 Thread Caleb Rackliffe (Jira)


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

Caleb Rackliffe updated CASSANDRA-16957:

Reviewers: Caleb Rackliffe
   Status: Review In Progress  (was: Patch Available)

> Actively update auth caches in the background
> -
>
> Key: CASSANDRA-16957
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16957
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Feature/Authorization
>Reporter: Josh McKenzie
>Assignee: Josh McKenzie
>Priority: Normal
> Fix For: 4.x
>
>
> Currently the guava cache backing the various auth caches refreshes its data 
> lazily; you won't get an update on cached credentials until you try and read 
> them and they're expired. For the PasswordCache in particular, this not only 
> gives us a window of async "serve the old while you fetch the new" which 
> isn't ideal, but also causes the cache to be invalidated and thus not adding 
> value / perf after expiration period (24h I believe by default).
> The expected behavior after this change is for the caches to auto-refresh 
> themselves on an interval so you a) don't have stale data sitting around 
> waiting to be served, and b) getting invalidated so having effectively a dead 
> cache for intermittent users assuming you have the backing resources to serve 
> them proactively.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer commented on CASSANDRA-17117:
-

Here are all the workflows in Circle CI - I've approved all of the pre-commit 
ones - will take time to work through the queue I'm sure.

Trunk 
JDK11 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/3/workflows/581de388-4488-4789-8880-95d7faab7877
JDK8 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/3/workflows/475b6807-199c-4cf0-a9e6-e2b63fc4e65d

4.0
JDK11 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/2/workflows/3efa087e-75e8-4794-99be-64c6475e3cab
JDK8 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/2/workflows/81e8a3ac-96ff-46a6-9c4c-0f5e68bdb276

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Sam Tunnicliffe
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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] [Assigned] (CASSANDRA-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer reassigned CASSANDRA-17117:
---

Assignee: Sam Tunnicliffe  (was: Doug Rohrer)

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Sam Tunnicliffe
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Brandon Williams (Jira)


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

Brandon Williams commented on CASSANDRA-17117:
--

bq. It looks like your 4.0 branch just didn't even start

I ninja-fixed that right before you commented. :)

bq. I can kick off the complete pre-commit Circle CI builds if that's useful at 
this point - just let me know.

Yes please, more CI never hurts but things are a bit delicate right now so it's 
especially good to have.

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer commented on CASSANDRA-17117:
-

Brandon:

Thanks for doing that - I do have some Circle runs with just unit tests run at 
the moment (4.0 passed unit tests, trunk had a few unrelated failures). It 
looks like your 4.0 branch just didn't even start running unfortunately.

* 4.0/jdk8 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/2/workflows/8ef3122d-1901-4099-a2cb-4ba56acd26b7/jobs/303
4.0 jdk11 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/2/workflows/e8061097-7f5c-4f12-a483-db4945d2c87c/jobs/301

Trunk/jdk8 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/3/workflows/3c6d730f-c72d-4343-bcf5-d97f3ddd3196/jobs/308
Trunk/jdk11 - 
https://app.circleci.com/pipelines/github/JeetKunDoug/cassandra/3/workflows/eb8566a6-0136-4f4e-b2c3-cd5d0bd6f0cb/jobs/307

I can kick off the complete pre-commit Circle CI builds if that's useful at 
this point - just let me know.

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Brandon Williams (Jira)


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

Brandon Williams edited comment on CASSANDRA-17117 at 11/3/21, 9:18 PM:


Not sure if you planned to add circle results, but I broke the circle config 
out into another commit and started jenkins CI:

||Branch||CI||
|[4.0|https://github.com/driftx/cassandra/tree/CASSANDRA-17117-4.0]|[!https://ci-cassandra.apache.org/job/Cassandra-devbranch/1265/badge/icon!|https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/1265/pipeline]|
|[trunk|https://github.com/driftx/cassandra/tree/CASSANDRA-17117-trunk]|[!https://ci-cassandra.apache.org/job/Cassandra-devbranch/1264/badge/icon!|https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/1264/pipeline]|



was (Author: brandon.williams):
Not sure if you planned to add circle results, but I broke the circle config 
out into another commit and started jenkins CI:

||Branch||CI||
|[4.0|https://github.com/driftx/cassandra/tree/CASSANDRA-17117-4.0]|[!https://ci-cassandra.apache.org/job/Cassandra-devbranch/1263/badge/icon!|https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/1263/pipeline]|
|[trunk|https://github.com/driftx/cassandra/tree/CASSANDRA-17117-trunk]|[!https://ci-cassandra.apache.org/job/Cassandra-devbranch/1264/badge/icon!|https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/1264/pipeline]|


> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Brandon Williams (Jira)


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

Brandon Williams commented on CASSANDRA-17117:
--

Not sure if you planned to add circle results, but I broke the circle config 
out into another commit and started jenkins CI:

||Branch||CI||
|[4.0|https://github.com/driftx/cassandra/tree/CASSANDRA-17117-4.0]|[!https://ci-cassandra.apache.org/job/Cassandra-devbranch/1263/badge/icon!|https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/1263/pipeline]|
|[trunk|https://github.com/driftx/cassandra/tree/CASSANDRA-17117-trunk]|[!https://ci-cassandra.apache.org/job/Cassandra-devbranch/1264/badge/icon!|https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/1264/pipeline]|


> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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] (CASSANDRASC-32) Sidecar health checks are failing since CassandraAdaptorDelegate is not started

2021-11-03 Thread Yifan Cai (Jira)


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

Yifan Cai updated CASSANDRASC-32:
-
Authors: Saranya Krishnakumar
  Fix Version/s: 1.0
Source Control Link: 
https://github.com/apache/cassandra-sidecar/commit/01d8b9c1893a37ae9c905f1844ca911c128ef6f4
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

Committed into trunk as 
[01d8b9c|https://github.com/apache/cassandra-sidecar/commit/01d8b9c1893a37ae9c905f1844ca911c128ef6f4]

> Sidecar health checks are failing since CassandraAdaptorDelegate is not 
> started
> ---
>
> Key: CASSANDRASC-32
> URL: https://issues.apache.org/jira/browse/CASSANDRASC-32
> Project: Sidecar for Apache Cassandra
>  Issue Type: Bug
>  Components: Rest API
>Reporter: Saranya Krishnakumar
>Assignee: Saranya Krishnakumar
>Priority: Normal
> Fix For: 1.0
>
>
> CassandraAdaptorDelegate class in Sidecar periodically checks if it is able 
> to connect to Cassandra instance. Currently we are not starting this delegate 
> and hence Sidecar health checks are failing. We need to start the delegate 
> while starting the server
>  
> https://github.com/apache/cassandra-sidecar/pull/24



--
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] (CASSANDRASC-32) Sidecar health checks are failing since CassandraAdaptorDelegate is not started

2021-11-03 Thread ASF subversion and git services (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRASC-32?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17438292#comment-17438292
 ] 

ASF subversion and git services commented on CASSANDRASC-32:


Commit 01d8b9c1893a37ae9c905f1844ca911c128ef6f4 in cassandra-sidecar's branch 
refs/heads/trunk from Saranya Krishnakumar
[ https://gitbox.apache.org/repos/asf?p=cassandra-sidecar.git;h=01d8b9c ]

Call the start method of CassandraAdaptorDelegate to start periodic health checl

patch by Saranya Krishnakumar; reviewed by Yifan Cai, Dinesh Joshi for 
CASSANDRASC-32


> Sidecar health checks are failing since CassandraAdaptorDelegate is not 
> started
> ---
>
> Key: CASSANDRASC-32
> URL: https://issues.apache.org/jira/browse/CASSANDRASC-32
> Project: Sidecar for Apache Cassandra
>  Issue Type: Bug
>  Components: Rest API
>Reporter: Saranya Krishnakumar
>Assignee: Saranya Krishnakumar
>Priority: Normal
>
> CassandraAdaptorDelegate class in Sidecar periodically checks if it is able 
> to connect to Cassandra instance. Currently we are not starting this delegate 
> and hence Sidecar health checks are failing. We need to start the delegate 
> while starting the server
>  
> https://github.com/apache/cassandra-sidecar/pull/24



--
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



[cassandra-sidecar] branch trunk updated (4d88af6 -> 01d8b9c)

2021-11-03 Thread ycai
This is an automated email from the ASF dual-hosted git repository.

ycai pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-sidecar.git.


from 4d88af6  Avoid having sidecar's health response code depend on 
Cassandra's health information
 add 01d8b9c  Call the start method of CassandraAdaptorDelegate to start 
periodic health checl

No new revisions were added by this update.

Summary of changes:
 .../cassandra/sidecar/common/CQLSession.java   |  5 +-
 .../sidecar/common/CassandraAdapterDelegate.java   | 59 +-
 .../cassandra/sidecar/CassandraSidecarDaemon.java  |  7 ++-
 3 files changed, 54 insertions(+), 17 deletions(-)

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



[jira] [Updated] (CASSANDRA-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer updated CASSANDRA-17117:

Test and Documentation Plan: Patches for 4.0 and trunk are attached - added 
an additional unit test to cover the backward-compatible and current cases to 
make sure both are still supported.
 Status: Patch Available  (was: In Progress)

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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] [Assigned] (CASSANDRA-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer reassigned CASSANDRA-17117:
---

Assignee: Doug Rohrer

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Assignee: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer updated CASSANDRA-17117:

Attachment: CASSANDRA-17117-trunk.patch

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch, CASSANDRA-17117-trunk.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17116) When zero-copy-streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread Dinesh Joshi (Jira)


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

Dinesh Joshi updated CASSANDRA-17116:
-
Reviewers: Dinesh Joshi

> When zero-copy-streaming sees a channel close this triggers the disk failure 
> policy
> ---
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17116) When zero-copy-streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17116:
--
Summary: When zero-copy-streaming sees a channel close this triggers the 
disk failure policy  (was: When streaming sees a channel close this triggers 
the disk failure policy)

> When zero-copy-streaming sees a channel close this triggers the disk failure 
> policy
> ---
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17116) When streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17116:
--
Test and Documentation Plan: tests added
 Status: Patch Available  (was: In Progress)

> When streaming sees a channel close this triggers the disk failure policy
> -
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17116) When streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17116:
--
Summary: When streaming sees a channel close this triggers the disk failure 
policy  (was: When zero-copy-streaming sees a channel close this triggers the 
disk failure policy)

> When streaming sees a channel close this triggers the disk failure policy
> -
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17085) commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread David Capwell (Jira)


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

David Capwell updated CASSANDRA-17085:
--
  Fix Version/s: (was: 4.x)
 4.1
  Since Version: NA
Source Control Link: 
https://github.com/apache/cassandra/commit/a540afd12ae4c29f15c54330242efdf1cf61354c
 Resolution: Fixed
 Status: Resolved  (was: Ready to Commit)

> commit log was switched from non-daemon to daemon threads, which causes the 
> JVM to exit in some case as no non-daemon threads are active
> 
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest/python
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.1
>
>
> Right now bootstrap tests are failing every time we run, this work is to 
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> >   node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost', 
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> process = 
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
> 
> if rc != 0:
> >   raise ToolError(cmd_args, rc, out, err)
> E   ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost', 
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit 
> status: 1; 
> E   stderr: nodetool: Failed to connect to 'localhost:7300' - 
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> >   node1.start()
> bootstrap_test.py:483: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in 
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, 
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n 
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> >   raise TimeoutError.create(start, timeout, msg, node)
> E   ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after 
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in 
> system.log:
> EHead: 
> ETail: ...
> {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



[cassandra] branch trunk updated: commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread dcapwell
This is an automated email from the ASF dual-hosted git repository.

dcapwell pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 111e94a  commit log was switched from non-daemon to daemon threads, 
which causes the JVM to exit in some case as no non-daemon threads are active
111e94a is described below

commit 111e94ae13381ede97de190d9e1af9a77cac2b21
Author: David Capwell 
AuthorDate: Wed Nov 3 11:53:36 2021 -0700

commit log was switched from non-daemon to daemon threads, which causes the 
JVM to exit in some case as no non-daemon threads are active

patch by David Capwell, Sam Tunnicliffe; reviewed by Sam Tunnicliffe for 
CASSANDRA-17085
---
 CHANGES.txt|  1 +
 .../cassandra/concurrent/ExecutorFactory.java  | 59 
 .../cassandra/concurrent/InfiniteLoopExecutor.java | 33 
 .../commitlog/AbstractCommitLogSegmentManager.java | 63 +++---
 .../db/commitlog/AbstractCommitLogService.java | 37 -
 .../org/apache/cassandra/utils/concurrent/Ref.java |  4 +-
 .../apache/cassandra/utils/memory/BufferPool.java  |  3 +-
 .../utils/memory/MemtableCleanerThread.java|  3 +-
 .../concurrent/InfiniteLoopExecutorTest.java   |  6 ++-
 9 files changed, 123 insertions(+), 86 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 32698f8..d1f5e73 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * commit log was switched from non-daemon to daemon threads, which causes the 
JVM to exit in some case as no non-daemon threads are active (CASSANDRA-17085)
  * Add a Denylist to block reads and writes on specific partition keys 
(CASSANDRA-12106)
  * v4+ protocol did not clean up client warnings, which caused leaking the 
state (CASSANDRA-17054)
  * Remove duplicate toCQLString in ReadCommand (CASSANDRA-17023)
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java 
b/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
index f1acd55..83f48d1 100644
--- a/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorFactory.java
@@ -18,12 +18,15 @@
 
 package org.apache.cassandra.concurrent;
 
-import java.util.function.Consumer;
-
+import org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon;
+import org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts;
+import org.apache.cassandra.concurrent.InfiniteLoopExecutor.SimulatorSafe;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Shared;
 
 import static java.lang.Thread.*;
+import static 
org.apache.cassandra.concurrent.InfiniteLoopExecutor.Daemon.DAEMON;
+import static 
org.apache.cassandra.concurrent.InfiniteLoopExecutor.Interrupts.UNSYNCHRONIZED;
 import static org.apache.cassandra.concurrent.NamedThreadFactory.createThread;
 import static org.apache.cassandra.concurrent.NamedThreadFactory.setupThread;
 import static 
org.apache.cassandra.concurrent.ThreadPoolExecutorBuilder.pooledJmx;
@@ -99,20 +102,21 @@ public interface ExecutorFactory extends 
ExecutorBuilderFactory.Jmxable interruptHandler);
+Interruptible infiniteLoop(String name, Interruptible.Task task, 
SimulatorSafe simulatorSafe, Daemon daemon, Interrupts interrupts);
 
 /**
  * Create and start a new InfiniteLoopExecutor to repeatedly invoke {@code 
runnable}.
@@ -133,11 +142,12 @@ public interface ExecutorFactory extends 
ExecutorBuilderFactory.Jmxable 
configureSequential(String name)
 {
 return 
ThreadPoolExecutorBuilder.sequential(SingleThreadExecutorPlus::new, 
contextClassLoader, threadGroup, uncaughtExceptionHandler, name);
 }
 
+@Override
 public ExecutorBuilder configurePooled(String 
name, int threads)
 {
 return 
ThreadPoolExecutorBuilder.pooled(ThreadPoolExecutorPlus::new, 
contextClassLoader, threadGroup, uncaughtExceptionHandler, name, threads);
 }
 
+@Override
 public ScheduledExecutorPlus scheduled(boolean executeOnShutdown, 
String name, int priority)
 {
 ScheduledThreadPoolExecutorPlus executor = new 
ScheduledThreadPoolExecutorPlus(newThreadFactory(name, priority));
@@ -243,22 +257,21 @@ public interface ExecutorFactory extends 
ExecutorBuilderFactory.Jmxable interruptHandler)
+public Interruptible infiniteLoop(String name, Interruptible.Task 
task, SimulatorSafe simulatorSafe, Daemon daemon, Interrupts interrupts)
 {
-return new InfiniteLoopExecutor(this, name, task, 
interruptHandler);
+return new InfiniteLoopExecutor(this, name, task, daemon, 
interrupts);
 }
 
 @Override
diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java 
b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.jav

[jira] [Commented] (CASSANDRA-17085) commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-17085:
---

Starting commit

CI Results (pending):
||Branch||Source||Circle CI||Jenkins||
|trunk|[branch|https://github.com/dcapwell/cassandra/tree/commit_remote_branch/CASSANDRA-17085-trunk-E65F2954-175A-46A4-B954-FC6A91109F61]|[build|https://app.circleci.com/pipelines/github/dcapwell/cassandra?branch=commit_remote_branch%2FCASSANDRA-17085-trunk-E65F2954-175A-46A4-B954-FC6A91109F61]|[build|https://ci-cassandra.apache.org/job/Cassandra-devbranch/1262/]|


> commit log was switched from non-daemon to daemon threads, which causes the 
> JVM to exit in some case as no non-daemon threads are active
> 
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest/python
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Right now bootstrap tests are failing every time we run, this work is to 
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> >   node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost', 
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> process = 
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
> 
> if rc != 0:
> >   raise ToolError(cmd_args, rc, out, err)
> E   ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost', 
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit 
> status: 1; 
> E   stderr: nodetool: Failed to connect to 'localhost:7300' - 
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> >   node1.start()
> bootstrap_test.py:483: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in 
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, 
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n 
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> >   raise TimeoutError.create(start, timeout, msg, node)
> E   ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after 
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in 
> system.log:
> EHead: 
> ETail: ...
> {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] [Comment Edited] (CASSANDRA-17031) Add support for PEM based key material for SSL

2021-11-03 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic edited comment on CASSANDRA-17031 at 11/3/21, 6:57 PM:
-

I will do it this Friday.


was (Author: stefan.miklosovic):
I will do it on this Friday.

> Add support for PEM based key material for SSL
> --
>
> Key: CASSANDRA-17031
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17031
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Messaging/Internode
>Reporter: Maulin Vasavada
>Assignee: Maulin Vasavada
>Priority: Normal
>
> h1. Scope
> Currently Cassandra supports standard keystore types for SSL 
> keys/certificates. The scope of this enhancement is to add support for PEM 
> based key material (keys/certificate) given that PEM is widely used common 
> format for the same. We intend to add support for Unencrypted and Password 
> Based Encrypted (PBE) PKCS#8 formatted Private Keys in PEM format with 
> standard algorithms (RSA, DSA and EC) along with the certificate chain for 
> the private key and PEM based X509 certificates. The work here is going to be 
> built on top of [CEP-9: Make SSLContext creation 
> pluggable|https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-9%3A+Make+SSLContext+creation+pluggable]
>  for which the code is merged for Apache Cassandra 4.1 release.
> We intend to support the key material be configured as direct PEM values 
> input OR via the file (configured with keystore and truststore configurations 
> today). We are not going to model PEM as a valid 'store_type' given that 
> 'store_type' has a [specific 
> definition|https://docs.oracle.com/en/java/javase/11/security/java-cryptography-architecture-jca-reference-guide.html#GUID-AB51DEFD-5238-4F96-967F-082F6D34FBEA].
>  
> h1. Approach
> Create an implementation for 
> [ISslContextFactory|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/security/ISslContextFactory.java]
>  extending 
> [FileBasedSslContextFactory|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java]
>  implementation to add PEM formatted key/certificates.
> h1. Motivation
> PEM is a widely used format for encoding Private Keys and X.509 Certificates 
> and Apache Cassandra's current implementation lacks the support for 
> specifying the PEM formatted key material for SSL configurations. This means 
> operators have to re-create the key material to comply to the supported 
> formats (using key/trust store types - jks, pkcs12 etc) and deal with an 
> operational task for managing it. This is an operational overhead we can 
> avoid by supporting the PEM format making Apache Cassandra even more customer 
> friendly and drive more adoption.
> h1. Proposed Changes
>  # A new implementation for ISslContextFactory - PEMBasedSslContextFactory 
> with the following supported configuration
> {panel:title=New configurations}
> {panel}
> |{{encryption_options:  }}
>  {{}}{{ssl_context_factory:}}
>  {{}}{{class_name: 
> org.apache.cassandra.security.PEMBasedSslContextFactory}}
>  {{}}{{parameters:}}
>  {{  }}{{private_key:  certificate chain>}}
>  {{  }}{{private_key_password:  }}{{private}} {{key }}{{if}} {{it is encrypted>}}
>  {{  }}{{trusted_certificates: }}|
> *NOTE:* We could reuse 'keystore_password' instead of the 
> 'private_key_password'. However PEM encoded private key is not a 'keystore' 
> in itself hence it would be inappropriate to piggyback on that other than 
> avoid duplicating similar fields.
>  # The PEMBasedSslContextFactory will also support file based key material 
> (and the corresponding HOT Reloading based on file timestamp updates) for the 
> PEM format via existing  'keystore' and 'truststore' encryption options. 
> However in that case the 'truststore_password' configuration won't be used 
> since generally PEM formatted certificates for truststore don't get encrypted 
> with a password.
>  # The PEMBasedSslContextFactory will internally create PKCS12 keystore for 
> private key and the trusted certificates. However, this doesn't impact the 
> user of the implementation in anyway and it is mentioned for clarity only.
>  



--
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-17031) Add support for PEM based key material for SSL

2021-11-03 Thread Stefan Miklosovic (Jira)


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

Stefan Miklosovic commented on CASSANDRA-17031:
---

I will do it on this Friday.

> Add support for PEM based key material for SSL
> --
>
> Key: CASSANDRA-17031
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17031
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Messaging/Internode
>Reporter: Maulin Vasavada
>Assignee: Maulin Vasavada
>Priority: Normal
>
> h1. Scope
> Currently Cassandra supports standard keystore types for SSL 
> keys/certificates. The scope of this enhancement is to add support for PEM 
> based key material (keys/certificate) given that PEM is widely used common 
> format for the same. We intend to add support for Unencrypted and Password 
> Based Encrypted (PBE) PKCS#8 formatted Private Keys in PEM format with 
> standard algorithms (RSA, DSA and EC) along with the certificate chain for 
> the private key and PEM based X509 certificates. The work here is going to be 
> built on top of [CEP-9: Make SSLContext creation 
> pluggable|https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-9%3A+Make+SSLContext+creation+pluggable]
>  for which the code is merged for Apache Cassandra 4.1 release.
> We intend to support the key material be configured as direct PEM values 
> input OR via the file (configured with keystore and truststore configurations 
> today). We are not going to model PEM as a valid 'store_type' given that 
> 'store_type' has a [specific 
> definition|https://docs.oracle.com/en/java/javase/11/security/java-cryptography-architecture-jca-reference-guide.html#GUID-AB51DEFD-5238-4F96-967F-082F6D34FBEA].
>  
> h1. Approach
> Create an implementation for 
> [ISslContextFactory|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/security/ISslContextFactory.java]
>  extending 
> [FileBasedSslContextFactory|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java]
>  implementation to add PEM formatted key/certificates.
> h1. Motivation
> PEM is a widely used format for encoding Private Keys and X.509 Certificates 
> and Apache Cassandra's current implementation lacks the support for 
> specifying the PEM formatted key material for SSL configurations. This means 
> operators have to re-create the key material to comply to the supported 
> formats (using key/trust store types - jks, pkcs12 etc) and deal with an 
> operational task for managing it. This is an operational overhead we can 
> avoid by supporting the PEM format making Apache Cassandra even more customer 
> friendly and drive more adoption.
> h1. Proposed Changes
>  # A new implementation for ISslContextFactory - PEMBasedSslContextFactory 
> with the following supported configuration
> {panel:title=New configurations}
> {panel}
> |{{encryption_options:  }}
>  {{}}{{ssl_context_factory:}}
>  {{}}{{class_name: 
> org.apache.cassandra.security.PEMBasedSslContextFactory}}
>  {{}}{{parameters:}}
>  {{  }}{{private_key:  certificate chain>}}
>  {{  }}{{private_key_password:  }}{{private}} {{key }}{{if}} {{it is encrypted>}}
>  {{  }}{{trusted_certificates: }}|
> *NOTE:* We could reuse 'keystore_password' instead of the 
> 'private_key_password'. However PEM encoded private key is not a 'keystore' 
> in itself hence it would be inappropriate to piggyback on that other than 
> avoid duplicating similar fields.
>  # The PEMBasedSslContextFactory will also support file based key material 
> (and the corresponding HOT Reloading based on file timestamp updates) for the 
> PEM format via existing  'keystore' and 'truststore' encryption options. 
> However in that case the 'truststore_password' configuration won't be used 
> since generally PEM formatted certificates for truststore don't get encrypted 
> with a password.
>  # The PEMBasedSslContextFactory will internally create PKCS12 keystore for 
> private key and the trusted certificates. However, this doesn't impact the 
> user of the implementation in anyway and it is mentioned for clarity only.
>  



--
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-16349) SSTableLoader reports error when SSTable(s) do not have data for some nodes

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova edited comment on CASSANDRA-16349 at 11/3/21, 6:23 PM:
---

[~Ge] I already rebased and backported prior submitting the tests. Please 
check the links in my previous comments


was (Author: e.dimitrova):
[~Ge] I already rebased and backported prior submitting the tests.

> SSTableLoader reports error when SSTable(s) do not have data for some nodes
> ---
>
> Key: CASSANDRA-16349
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16349
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Serban Teodorescu
>Assignee: Serban Teodorescu
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Running SSTableLoader in verbose mode will show error(s) if there are node(s) 
> that do not own any data from the SSTable(s). This can happen in at least 2 
> cases:
>  # SSTableLoader is used to stream backups while keeping the same token ranges
>  # SSTable(s) are created with CQLSSTableWriter to match token ranges (this 
> can bring better performance by using ZeroCopy streaming)
> Partial output of the SSTableLoader:
> {quote}ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] 
> Remote peer /127.0.0.4:7000 failed stream session.
> ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] Remote peer 
> /127.0.0.3:7000 failed stream session.
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.515KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.427KiB/s)
> {quote}
>  
> Stack trace:
> {quote}java.util.concurrent.ExecutionException: 
> org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:552)
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
> at org.apache.cassandra.tools.BulkLoader.load(BulkLoader.java:99)
> at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:49)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
> at 
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
> at 
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at 
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
> at 
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:505)
> at 
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:819)
> at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:595)
> at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
> at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.base/java.lang.Thread.run(Thread.java:844)
> {quote}
> To reproduce create a cluster with ccm with more nodes than the RF, put some 
> data into it copy a SSTable and stream it.
>  
> The error originates on the nodes, the following stack trace is shown in the 
> logs:
> {quote}java.lang.IllegalStateException: Stream hasn't been read yet
>     at 
> com.google.common.base.Preconditions.checkState(Preconditions.java:507)
>     at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.getSize(CassandraIncomingFile.java:96)
>     at 
> org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:789)
>     at 
> org.apache.cassandra.streaming.St

[jira] [Commented] (CASSANDRA-16349) SSTableLoader reports error when SSTable(s) do not have data for some nodes

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova commented on CASSANDRA-16349:
-

[~Ge] I already rebased and backported prior submitting the tests.

> SSTableLoader reports error when SSTable(s) do not have data for some nodes
> ---
>
> Key: CASSANDRA-16349
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16349
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Serban Teodorescu
>Assignee: Serban Teodorescu
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Running SSTableLoader in verbose mode will show error(s) if there are node(s) 
> that do not own any data from the SSTable(s). This can happen in at least 2 
> cases:
>  # SSTableLoader is used to stream backups while keeping the same token ranges
>  # SSTable(s) are created with CQLSSTableWriter to match token ranges (this 
> can bring better performance by using ZeroCopy streaming)
> Partial output of the SSTableLoader:
> {quote}ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] 
> Remote peer /127.0.0.4:7000 failed stream session.
> ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] Remote peer 
> /127.0.0.3:7000 failed stream session.
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.515KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.427KiB/s)
> {quote}
>  
> Stack trace:
> {quote}java.util.concurrent.ExecutionException: 
> org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:552)
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
> at org.apache.cassandra.tools.BulkLoader.load(BulkLoader.java:99)
> at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:49)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
> at 
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
> at 
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at 
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
> at 
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:505)
> at 
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:819)
> at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:595)
> at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
> at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.base/java.lang.Thread.run(Thread.java:844)
> {quote}
> To reproduce create a cluster with ccm with more nodes than the RF, put some 
> data into it copy a SSTable and stream it.
>  
> The error originates on the nodes, the following stack trace is shown in the 
> logs:
> {quote}java.lang.IllegalStateException: Stream hasn't been read yet
>     at 
> com.google.common.base.Preconditions.checkState(Preconditions.java:507)
>     at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.getSize(CassandraIncomingFile.java:96)
>     at 
> org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:789)
>     at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:587)
>     at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
>   

[jira] [Commented] (CASSANDRA-16349) SSTableLoader reports error when SSTable(s) do not have data for some nodes

2021-11-03 Thread Aleksandr Sorokoumov (Jira)


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

Aleksandr Sorokoumov commented on CASSANDRA-16349:
--

Thank you for the review and running the tests [~e.dimitrova]! Tomorrow I will 
mark the dtest to run only since 3.0, rebase the patch against latest trunk and 
backport it to 4.0.

> SSTableLoader reports error when SSTable(s) do not have data for some nodes
> ---
>
> Key: CASSANDRA-16349
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16349
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Serban Teodorescu
>Assignee: Serban Teodorescu
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Running SSTableLoader in verbose mode will show error(s) if there are node(s) 
> that do not own any data from the SSTable(s). This can happen in at least 2 
> cases:
>  # SSTableLoader is used to stream backups while keeping the same token ranges
>  # SSTable(s) are created with CQLSSTableWriter to match token ranges (this 
> can bring better performance by using ZeroCopy streaming)
> Partial output of the SSTableLoader:
> {quote}ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] 
> Remote peer /127.0.0.4:7000 failed stream session.
> ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] Remote peer 
> /127.0.0.3:7000 failed stream session.
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.515KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.427KiB/s)
> {quote}
>  
> Stack trace:
> {quote}java.util.concurrent.ExecutionException: 
> org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:552)
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
> at org.apache.cassandra.tools.BulkLoader.load(BulkLoader.java:99)
> at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:49)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
> at 
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
> at 
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at 
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
> at 
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:505)
> at 
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:819)
> at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:595)
> at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
> at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.base/java.lang.Thread.run(Thread.java:844)
> {quote}
> To reproduce create a cluster with ccm with more nodes than the RF, put some 
> data into it copy a SSTable and stream it.
>  
> The error originates on the nodes, the following stack trace is shown in the 
> logs:
> {quote}java.lang.IllegalStateException: Stream hasn't been read yet
>     at 
> com.google.common.base.Preconditions.checkState(Preconditions.java:507)
>     at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.getSize(CassandraIncomingFile.java:96)
>     at 
> org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:789)
>     at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:587)
>     at 
> org.apache.cassandr

[jira] [Updated] (CASSANDRA-17085) commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread Josh McKenzie (Jira)


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

Josh McKenzie updated CASSANDRA-17085:
--
Attachment: (was: 4.0_iwork_p97.html)

> commit log was switched from non-daemon to daemon threads, which causes the 
> JVM to exit in some case as no non-daemon threads are active
> 
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest/python
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Right now bootstrap tests are failing every time we run, this work is to 
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> >   node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost', 
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> process = 
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
> 
> if rc != 0:
> >   raise ToolError(cmd_args, rc, out, err)
> E   ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost', 
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit 
> status: 1; 
> E   stderr: nodetool: Failed to connect to 'localhost:7300' - 
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> >   node1.start()
> bootstrap_test.py:483: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in 
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, 
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n 
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> >   raise TimeoutError.create(start, timeout, msg, node)
> E   ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after 
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in 
> system.log:
> EHead: 
> ETail: ...
> {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-17085) commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread Josh McKenzie (Jira)


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

Josh McKenzie updated CASSANDRA-17085:
--
Attachment: 4.0_iwork_p97.html

> commit log was switched from non-daemon to daemon threads, which causes the 
> JVM to exit in some case as no non-daemon threads are active
> 
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest/python
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
> Attachments: 4.0_iwork_p97.html
>
>
> Right now bootstrap tests are failing every time we run, this work is to 
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> >   node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost', 
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> process = 
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
> 
> if rc != 0:
> >   raise ToolError(cmd_args, rc, out, err)
> E   ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost', 
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit 
> status: 1; 
> E   stderr: nodetool: Failed to connect to 'localhost:7300' - 
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> >   node1.start()
> bootstrap_test.py:483: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in 
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, 
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n 
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> >   raise TimeoutError.create(start, timeout, msg, node)
> E   ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after 
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in 
> system.log:
> EHead: 
> ETail: ...
> {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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)


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

Doug Rohrer updated CASSANDRA-17117:

Attachment: CASSANDRA-17117-4.0.patch

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Priority: Normal
> Attachments: CASSANDRA-17117-4.0.patch
>
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17116) When zero-copy-streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell commented on CASSANDRA-17116:
---

have a patch which replicates the problem and found several other issues as 
well.  This issue only triggers disk policy for zero-copy-streaming but the 
error handling in streaming also was causing issues as it wouldn't properly 
close the session; leaving it hanging...

> When zero-copy-streaming sees a channel close this triggers the disk failure 
> policy
> ---
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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] [Assigned] (CASSANDRA-17116) When zero-copy-streaming sees a channel close this triggers the disk failure policy

2021-11-03 Thread David Capwell (Jira)


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

David Capwell reassigned CASSANDRA-17116:
-

Assignee: David Capwell

> When zero-copy-streaming sees a channel close this triggers the disk failure 
> policy
> ---
>
> Key: CASSANDRA-17116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17116
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Streaming
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Found in CASSANDRA-17085.
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256
> {code}
> ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 
> DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system 
> exception on startup, disk failure policy "stop"
> org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206)
>   at 
> org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125)
>   at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51)
>   at 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37)
>   at 
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50)
>   at 
> org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62)
>   at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.nio.channels.ClosedChannelException: null
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136)
>   at 
> org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155)
>   at 
> org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217)
>   ... 9 common frames omitted
> {code}
> When bootstrap fails and streaming is closed, this triggers the disk failure 
> policy which causes the JVM to halt by default (if this happens outside of 
> bootstrap, then we stop transports and keep the JVM up).
> org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle 
> this by ignoring this exception, but the call to 
> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize
>  Does try/catch and inspects exception; triggering this condition.



--
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-17070) ViewComplexTest hardening

2021-11-03 Thread Jira


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

Andres de la Peña commented on CASSANDRA-17070:
---

The PR for trunk looks good to me. Here are 200 runs of {{ViewComplex*Test}} in 
the PR for trunk:
 * 
[j8|https://app.circleci.com/pipelines/github/adelapena/cassandra/1114/workflows/2a6ea982-7109-4cf2-b5af-7f06188f2d4f]
 * 
[j11|https://app.circleci.com/pipelines/github/adelapena/cassandra/1114/workflows/12ab59cb-24d5-4795-a309-21b7eb6b187c]

If those succeed I think we'll be definitively ready to merge.

> ViewComplexTest hardening
> -
>
> Key: CASSANDRA-17070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17070
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/unit
>Reporter: Berenguer Blasi
>Assignee: Berenguer Blasi
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>
> I have seen a number of times already the {{ViewComplexTest}} family timeout 
> on test method teardown. This leaves a dirty env behind triggering the 
> following test methods to fail on it. This ticket aims at hardening them.



--
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-17085) commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread Sam Tunnicliffe (Jira)


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

Sam Tunnicliffe updated CASSANDRA-17085:

Status: Ready to Commit  (was: Review In Progress)

+1 LGTM

> commit log was switched from non-daemon to daemon threads, which causes the 
> JVM to exit in some case as no non-daemon threads are active
> 
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest/python
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Right now bootstrap tests are failing every time we run, this work is to 
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> >   node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost', 
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> process = 
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
> 
> if rc != 0:
> >   raise ToolError(cmd_args, rc, out, err)
> E   ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost', 
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit 
> status: 1; 
> E   stderr: nodetool: Failed to connect to 'localhost:7300' - 
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> >   node1.start()
> bootstrap_test.py:483: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in 
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, 
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n 
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> >   raise TimeoutError.create(start, timeout, msg, node)
> E   ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after 
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in 
> system.log:
> EHead: 
> ETail: ...
> {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-17085) commit log was switched from non-daemon to daemon threads, which causes the JVM to exit in some case as no non-daemon threads are active

2021-11-03 Thread Sam Tunnicliffe (Jira)


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

Sam Tunnicliffe updated CASSANDRA-17085:

Reviewers: Sam Tunnicliffe, Sam Tunnicliffe  (was: Sam Tunnicliffe)
   Sam Tunnicliffe, Sam Tunnicliffe
   Status: Review In Progress  (was: Patch Available)

> commit log was switched from non-daemon to daemon threads, which causes the 
> JVM to exit in some case as no non-daemon threads are active
> 
>
> Key: CASSANDRA-17085
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17085
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/dtest/python
>Reporter: David Capwell
>Assignee: David Capwell
>Priority: Normal
> Fix For: 4.x
>
>
> Right now bootstrap tests are failing every time we run, this work is to 
> debug and fix the underling issue.
> Examples:
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7089
> {code}
> >   node3.nodetool('bootstrap resume')
> bootstrap_test.py:1014: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:1005: in nodetool
> return handle_external_tool_process(p, ['nodetool', '-h', 'localhost', 
> '-p', str(self.jmx_port)] + shlex.split(cmd))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> process = 
> cmd_args = ['nodetool', '-h', 'localhost', '-p', '7300', 'bootstrap', ...]
> def handle_external_tool_process(process, cmd_args):
> out, err = process.communicate()
> if (out is not None) and isinstance(out, bytes):
> out = out.decode()
> if (err is not None) and isinstance(err, bytes):
> err = err.decode()
> rc = process.returncode
> 
> if rc != 0:
> >   raise ToolError(cmd_args, rc, out, err)
> E   ccmlib.node.ToolError: Subprocess ['nodetool', '-h', 'localhost', 
> '-p', '7300', 'bootstrap', 'resume'] exited with non-zero status; exit 
> status: 1; 
> E   stderr: nodetool: Failed to connect to 'localhost:7300' - 
> EOFException: 'null'.
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:2305: ToolError
> {code}
> https://app.circleci.com/pipelines/github/dcapwell/cassandra/1062/workflows/ba3e6395-ef22-4724-8424-0549e65d8cff/jobs/7087
> {code}
> >   node1.start()
> bootstrap_test.py:483: 
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:895: in start
> node.watch_log_for_alive(self, from_mark=mark)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:664: in 
> watch_log_for_alive
> self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, 
> filename=filename)
> ../env3.6/lib/python3.6/site-packages/ccmlib/node.py:592: in watch_log_for
> head=reads[:50], tail="..."+reads[len(reads)-150:]))
> _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ 
> _ 
> start = 1635453190.3118386, timeout = 120
> msg = "Missing: ['127.0.0.1:7000.* is now UP'] not found in system.log:\n 
> Head: \n Tail: ..."
> node = 'node3'
> @staticmethod
> def raise_if_passed(start, timeout, msg, node=None):
> if start + timeout < time.time():
> >   raise TimeoutError.create(start, timeout, msg, node)
> E   ccmlib.node.TimeoutError: 28 Oct 2021 20:35:10 [node3] after 
> 120.12/120 seconds Missing: ['127.0.0.1:7000.* is now UP'] not found in 
> system.log:
> EHead: 
> ETail: ...
> {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-16349) SSTableLoader reports error when SSTable(s) do not have data for some nodes

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova commented on CASSANDRA-16349:
-

[4.0|https://github.com/ekaterinadimitrova2/cassandra/pull/new/16349-v4.0] | 
[Jenkins CI run|https://jenkins-cm4.apache.org/job/Cassandra-devbranch/1261/]

> SSTableLoader reports error when SSTable(s) do not have data for some nodes
> ---
>
> Key: CASSANDRA-16349
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16349
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Serban Teodorescu
>Assignee: Serban Teodorescu
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Running SSTableLoader in verbose mode will show error(s) if there are node(s) 
> that do not own any data from the SSTable(s). This can happen in at least 2 
> cases:
>  # SSTableLoader is used to stream backups while keeping the same token ranges
>  # SSTable(s) are created with CQLSSTableWriter to match token ranges (this 
> can bring better performance by using ZeroCopy streaming)
> Partial output of the SSTableLoader:
> {quote}ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] 
> Remote peer /127.0.0.4:7000 failed stream session.
> ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] Remote peer 
> /127.0.0.3:7000 failed stream session.
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.515KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.427KiB/s)
> {quote}
>  
> Stack trace:
> {quote}java.util.concurrent.ExecutionException: 
> org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:552)
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
> at org.apache.cassandra.tools.BulkLoader.load(BulkLoader.java:99)
> at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:49)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
> at 
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
> at 
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at 
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
> at 
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:505)
> at 
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:819)
> at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:595)
> at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
> at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.base/java.lang.Thread.run(Thread.java:844)
> {quote}
> To reproduce create a cluster with ccm with more nodes than the RF, put some 
> data into it copy a SSTable and stream it.
>  
> The error originates on the nodes, the following stack trace is shown in the 
> logs:
> {quote}java.lang.IllegalStateException: Stream hasn't been read yet
>     at 
> com.google.common.base.Preconditions.checkState(Preconditions.java:507)
>     at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.getSize(CassandraIncomingFile.java:96)
>     at 
> org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:789)
>     at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:587)
>     at 
> org.apache.cassandra.streaming.async.Streaming

[jira] [Assigned] (CASSANDRA-14466) Enable Direct I/O

2021-11-03 Thread Mulugeta Mammo (Jira)


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

Mulugeta Mammo reassigned CASSANDRA-14466:
--

Assignee: (was: Mulugeta Mammo)

> Enable Direct I/O 
> --
>
> Key: CASSANDRA-14466
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14466
> Project: Cassandra
>  Issue Type: New Feature
>  Components: Legacy/Local Write-Read Paths
>Reporter: Mulugeta Mammo
>Priority: Normal
> Fix For: 4.x
>
>
> Hi,
> JDK 10 introduced a new API for Direct IO that enables applications to bypass 
> the file system cache and potentially improve performance. Details of this 
> feature can be found at [https://bugs.openjdk.java.net/browse/JDK-8164900].
> This patch uses the JDK 10 API to enable Direct IO for the Cassandra read 
> path. By default, we have disabled this feature; but it can be enabled using 
> a  new configuration parameter, enable_direct_io_for_read_path. We have 
> conducted a Cassandra read-only stress test and measured a throughput gain of 
> up to 60% on flash drives.
> The patch requires JDK 10 Cassandra Support - 
> https://issues.apache.org/jira/browse/CASSANDRA-9608 
> Please review the patch and let us know your feedback.
> Thanks,
> [^direct_io.patch]
>  



--
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-16349) SSTableLoader reports error when SSTable(s) do not have data for some nodes

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova edited comment on CASSANDRA-16349 at 11/3/21, 3:47 PM:
---

I think we need to mark the dtests to be since 3.0. Even if 2.2 is out of 
support, we still have it in CI and there might be third parties who still 
support it.

I can confirm that the test passes with 3.0 and 3.11 locally. 

As apparently a release happened somewhere around this work, I will backport 
the patch to 4.0 branch too. Still don't have time for proper review so I will 
post whatever I have time for and leave it to NEEDS COMMITTER status in case 
someone has cycles before me.

[trunk|https://github.com/ekaterinadimitrova2/cassandra/pull/new/16349-streaming-sstableloader-4.0]
 | 
[dtest|https://github.com/ekaterinadimitrova2/cassandra-dtest/pull/new/CASSANDRA-16349]
 | [Jenkins CI |https://jenkins-cm4.apache.org/job/Cassandra-devbranch/1260/]


was (Author: e.dimitrova):
[trunk|https://github.com/ekaterinadimitrova2/cassandra/pull/new/16349-streaming-sstableloader-4.0]
 | 
[dtest|https://github.com/ekaterinadimitrova2/cassandra-dtest/pull/new/CASSANDRA-16349]
 | [Jenkins CI | https://jenkins-cm4.apache.org/job/Cassandra-devbranch/1260/]

> SSTableLoader reports error when SSTable(s) do not have data for some nodes
> ---
>
> Key: CASSANDRA-16349
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16349
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Serban Teodorescu
>Assignee: Serban Teodorescu
>Priority: Normal
> Fix For: 4.0.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Running SSTableLoader in verbose mode will show error(s) if there are node(s) 
> that do not own any data from the SSTable(s). This can happen in at least 2 
> cases:
>  # SSTableLoader is used to stream backups while keeping the same token ranges
>  # SSTable(s) are created with CQLSSTableWriter to match token ranges (this 
> can bring better performance by using ZeroCopy streaming)
> Partial output of the SSTableLoader:
> {quote}ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] 
> Remote peer /127.0.0.4:7000 failed stream session.
> ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] Remote peer 
> /127.0.0.3:7000 failed stream session.
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.515KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.427KiB/s)
> {quote}
>  
> Stack trace:
> {quote}java.util.concurrent.ExecutionException: 
> org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:552)
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
> at org.apache.cassandra.tools.BulkLoader.load(BulkLoader.java:99)
> at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:49)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
> at 
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
> at 
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at 
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
> at 
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:505)
> at 
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:819)
> at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:595)
> at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$Stream

[jira] [Updated] (CASSANDRA-16349) SSTableLoader reports error when SSTable(s) do not have data for some nodes

2021-11-03 Thread Ekaterina Dimitrova (Jira)


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

Ekaterina Dimitrova updated CASSANDRA-16349:

Fix Version/s: 4.x

> SSTableLoader reports error when SSTable(s) do not have data for some nodes
> ---
>
> Key: CASSANDRA-16349
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16349
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/sstable
>Reporter: Serban Teodorescu
>Assignee: Serban Teodorescu
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> Running SSTableLoader in verbose mode will show error(s) if there are node(s) 
> that do not own any data from the SSTable(s). This can happen in at least 2 
> cases:
>  # SSTableLoader is used to stream backups while keeping the same token ranges
>  # SSTable(s) are created with CQLSSTableWriter to match token ranges (this 
> can bring better performance by using ZeroCopy streaming)
> Partial output of the SSTableLoader:
> {quote}ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] 
> Remote peer /127.0.0.4:7000 failed stream session.
> ERROR 02:47:47,842 [Stream #fa8e73b0-3da5-11eb-9c47-c5d27ae8fe47] Remote peer 
> /127.0.0.3:7000 failed stream session.
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.611KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.515KiB/s)
> progress: [/127.0.0.4:7000]0:0/1 100% [/127.0.0.3:7000]0:0/1 100% 
> [/127.0.0.2:7000]0:7/7 100% [/127.0.0.1:7000]0:7/7 100% total: 100% 
> 0.000KiB/s (avg: 1.427KiB/s)
> {quote}
>  
> Stack trace:
> {quote}java.util.concurrent.ExecutionException: 
> org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:552)
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
> at org.apache.cassandra.tools.BulkLoader.load(BulkLoader.java:99)
> at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:49)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
> at 
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
> at 
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
> at 
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
> at 
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:505)
> at 
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:819)
> at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:595)
> at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
> at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.base/java.lang.Thread.run(Thread.java:844)
> {quote}
> To reproduce create a cluster with ccm with more nodes than the RF, put some 
> data into it copy a SSTable and stream it.
>  
> The error originates on the nodes, the following stack trace is shown in the 
> logs:
> {quote}java.lang.IllegalStateException: Stream hasn't been read yet
>     at 
> com.google.common.base.Preconditions.checkState(Preconditions.java:507)
>     at 
> org.apache.cassandra.db.streaming.CassandraIncomingFile.getSize(CassandraIncomingFile.java:96)
>     at 
> org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:789)
>     at 
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:587)
>     at 
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
>     at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>     at

[jira] [Updated] (CASSANDRA-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Brandon Williams (Jira)


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

Brandon Williams updated CASSANDRA-17117:
-
 Bug Category: Parent values: Code(13163)
   Complexity: Normal
Discovered By: User Report
 Severity: Low
   Status: Open  (was: Triage Needed)

> CQLSSTableWriter backwards compatibility fix for Date fields
> 
>
> Key: CASSANDRA-17117
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/SSTable
>Reporter: Doug Rohrer
>Priority: Normal
>
> {{cassandra-all}} library consumers of the CQLSSTableWriter class cannot 
> easily create an appropriate instance of the {{LocalDate}} class in order to 
> write SSTables including dates, and the current implementation no longer 
> accepts a {{int}} value as a valid input as previous versions of the class 
> would - we used to use 
> {{((AbstractType)columnSpecification.type).decompose(value);}} in order to 
> serialize the value, but now we use the type codec's .serialize method. 
> Unfortunately, this doesn't work when the consumer can't easily create the 
> type needed (serialize, in the case of dates, takes a {{LocalDate}} instance 
> which is not easy to construct outside of Cassandra internal code).
> This can be worked around by catching the resulting {{ClassCastException}} 
> thrown from the {{serialize}} call and falling back to the older 
> {{decompose}} implementation, which would maintain backwards-compatibility 
> with other users of the CQLSSTableWriter in cases where they are passing 
> integers for date fields, which used to work.



--
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-17117) CQLSSTableWriter backwards compatibility fix for Date fields

2021-11-03 Thread Doug Rohrer (Jira)
Doug Rohrer created CASSANDRA-17117:
---

 Summary: CQLSSTableWriter backwards compatibility fix for Date 
fields
 Key: CASSANDRA-17117
 URL: https://issues.apache.org/jira/browse/CASSANDRA-17117
 Project: Cassandra
  Issue Type: Bug
  Components: Local/SSTable
Reporter: Doug Rohrer


{{cassandra-all}} library consumers of the CQLSSTableWriter class cannot easily 
create an appropriate instance of the {{LocalDate}} class in order to write 
SSTables including dates, and the current implementation no longer accepts a 
{{int}} value as a valid input as previous versions of the class would - we 
used to use {{((AbstractType)columnSpecification.type).decompose(value);}} in 
order to serialize the value, but now we use the type codec's .serialize 
method. Unfortunately, this doesn't work when the consumer can't easily create 
the type needed (serialize, in the case of dates, takes a {{LocalDate}} 
instance which is not easy to construct outside of Cassandra internal code).

This can be worked around by catching the resulting {{ClassCastException}} 
thrown from the {{serialize}} call and falling back to the older {{decompose}} 
implementation, which would maintain backwards-compatibility with other users 
of the CQLSSTableWriter in cases where they are passing integers for date 
fields, which used to work.



--
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-16801) PasswordObfuscator should not assume PASSWORD is the last item in the WITH clause

2021-11-03 Thread Benjamin Lerer (Jira)


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

Benjamin Lerer updated CASSANDRA-16801:
---
Status: Review In Progress  (was: Needs Committer)

> PasswordObfuscator should not assume PASSWORD is the last item in the WITH 
> clause
> -
>
> Key: CASSANDRA-16801
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16801
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/auditlogging
>Reporter: Caleb Rackliffe
>Assignee: Berenguer Blasi
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>
> CASSANDRA-16669 introduced support for obfuscating passwords for audit log 
> statements, but there are a few cases where the obfuscation logic can destroy 
> some of the contents of the original/provided string.
> ex. This is perfectly valid...
> {noformat}
> WITH LOGIN = false AND PASSWORD = 'bar' AND SUPERUSER = false
> {noformat}
> ...but calling obfuscate() on it will produce...
> {noformat}
> WITH LOGIN = false AND PASSWORD ***
> {noformat}
> -We should be able to create a reasonable RegEx and use String#replaceAll() 
> to both simplify and correct PasswordObfuscator#obfuscate().-



--
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-17073) The initial endpoint state is not propagated to all registered listeners in Gossiper

2021-11-03 Thread Jacek Lewandowski (Jira)


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

Jacek Lewandowski commented on CASSANDRA-17073:
---

I've implemented a failing test here 
https://github.com/apache/cassandra/pull/1300

I'm wondering whether it is valid

> The initial endpoint state is not propagated to all registered listeners in 
> Gossiper
> 
>
> Key: CASSANDRA-17073
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17073
> Project: Cassandra
>  Issue Type: Bug
>  Components: Local/Other
>Reporter: Jacek Lewandowski
>Assignee: Jacek Lewandowski
>Priority: Normal
>
> Extracted this as a separate ticket per discussion on CASSANDRA-17044
> Once joined the ring, {{StorageService}} artificially executes {{onChange}} 
> (endpoint event subscriber method) with all the updated values. However, what 
> we want probably that all registered endpoint event subscribers get updated.



--
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-17070) ViewComplexTest hardening

2021-11-03 Thread Berenguer Blasi (Jira)


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

Berenguer Blasi commented on CASSANDRA-17070:
-

I added the trunk PR with CI. There's no rush so this will give you time to 
take a look at it. Let me know if you're still ok I merge. 

> ViewComplexTest hardening
> -
>
> Key: CASSANDRA-17070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17070
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/unit
>Reporter: Berenguer Blasi
>Assignee: Berenguer Blasi
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>
> I have seen a number of times already the {{ViewComplexTest}} family timeout 
> on test method teardown. This leaves a dirty env behind triggering the 
> following test methods to fail on it. This ticket aims at hardening them.



--
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-16801) PasswordObfuscator should not assume PASSWORD is the last item in the WITH clause

2021-11-03 Thread Benjamin Lerer (Jira)


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

Benjamin Lerer updated CASSANDRA-16801:
---
Reviewers: Benjamin Lerer, Ekaterina Dimitrova

> PasswordObfuscator should not assume PASSWORD is the last item in the WITH 
> clause
> -
>
> Key: CASSANDRA-16801
> URL: https://issues.apache.org/jira/browse/CASSANDRA-16801
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tool/auditlogging
>Reporter: Caleb Rackliffe
>Assignee: Berenguer Blasi
>Priority: Normal
> Fix For: 4.0.x, 4.x
>
>
> CASSANDRA-16669 introduced support for obfuscating passwords for audit log 
> statements, but there are a few cases where the obfuscation logic can destroy 
> some of the contents of the original/provided string.
> ex. This is perfectly valid...
> {noformat}
> WITH LOGIN = false AND PASSWORD = 'bar' AND SUPERUSER = false
> {noformat}
> ...but calling obfuscate() on it will produce...
> {noformat}
> WITH LOGIN = false AND PASSWORD ***
> {noformat}
> -We should be able to create a reasonable RegEx and use String#replaceAll() 
> to both simplify and correct PasswordObfuscator#obfuscate().-



--
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