[jira] [Commented] (CASSANDRA-18110) Streaming fails during multiple concurrent host replacements
[ https://issues.apache.org/jira/browse/CASSANDRA-18110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654713#comment-17654713 ] David Capwell commented on CASSANDRA-18110: --- bq. For #3 - wouldn't disabling tracking mean that the virtual table isn't usable? Correct, this allows you to break the vtable if the feature causes things to not be stable. bq. I'm also in favor of #4. I am working on a patch doing both #3 and #4; so fix the issue by doing less work, and allow opt-out... bq. I was also thinking of changing the pacing (via debounce) for individual file progress events, so a progress event does not trigger handleStreamEvent if it was called within the last X millis. I don't think we have enough people familiar with Streaming to make that safe... Even the vtable was a problem as no one is around who knows streaming! Changing the pacing could have side effects non of us are aware of... > Streaming fails during multiple concurrent host replacements > > > Key: CASSANDRA-18110 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18110 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jon Meredith >Assignee: Jon Meredith >Priority: Normal > Fix For: 4.1.x > > > Running four concurrent host replacements on a 4.1.0 development cluster has > repeatably failed to complete bootstrap with all four hosts failing bootsrrap > and staying in JOINING, logging the message. > {code:java} > ERROR 2022-12-07T21:15:48,860 [main] > org.apache.cassandra.service.StorageService:2019 - Error while waiting on > bootstrap to complete. Bootstrap will have to be restarted. > {code} > Bootstrap fails as the the FileStreamTasks on the streaming followers > encounter an EOF while transmitting the files. > {code:java} > ERROR 2022-12-07T15:49:39,164 [NettyStreaming-Outbound-/1.2.3.4.7000:2] > org.apache.cassandra.streaming.StreamSession:718 - [Stream > #8d313690-7674-11ed-813f-95c261b64a82] Streaming error occurred on session > with peer 1.2.3.4:7000 through 1.2.3.4:40292 > org.apache.cassandra.net.AsyncChannelOutputPlus$FlushException: The channel > this output stream was writing to has been closed >at > org.apache.cassandra.net.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:158) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.waitForSpace(AsyncChannelOutputPlus.java:140) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.beginFlush(AsyncChannelOutputPlus.java:97) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.lambda$writeToChannel$0(AsyncStreamingOutputPlus.java:124) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.lambda$write$0(CassandraCompressedStreamWriter.java:89) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.writeToChannel(AsyncStreamingOutputPlus.java:120) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.write(CassandraCompressedStreamWriter.java:88) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:177) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage.serialize(OutgoingStreamMessage.java:87) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:45) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:34) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:39) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:311) > [cassandra.jar] >at > org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96) > [cassandra.jar] >at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61) > [cassandra.jar] >at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71) > [cassandra.jar] >at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > [?:?] >at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > [?:?] >at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) >
[jira] [Commented] (CASSANDRA-18100) Support CAS and serial read on Accord
[ https://issues.apache.org/jira/browse/CASSANDRA-18100?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654686#comment-17654686 ] Caleb Rackliffe commented on CASSANDRA-18100: - Just finished my first pass at review, and dropped a bunch of questions, nits, and suggestions. The {{Bound}} serialization work might make it possible for me to simplify some things in {{TxnCondition.Value}} in CASSANDRA-18107, but I won't know for sure until I dig in. > Support CAS and serial read on Accord > - > > Key: CASSANDRA-18100 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18100 > Project: Cassandra > Issue Type: Sub-task > Components: Accord >Reporter: Ariel Weisberg >Assignee: Ariel Weisberg >Priority: Normal > Time Spent: 8h 40m > Remaining Estimate: 0h > > This is a pre-requisite for live migration and roll back. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-18099) Use checked casts when reading vints as ints
[ https://issues.apache.org/jira/browse/CASSANDRA-18099?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-18099: --- Fix Version/s: 4.2 Resolution: Fixed Status: Resolved (was: Ready to Commit) Committed as [d7c5c547f025301780658b37e6e8a591bc4a5b36|https://github.com/apache/cassandra/commit/d7c5c547f025301780658b37e6e8a591bc4a5b36] > Use checked casts when reading vints as ints > > > Key: CASSANDRA-18099 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18099 > Project: Cassandra > Issue Type: Improvement > Components: Messaging/Client, Messaging/Internode, Messaging/Thrift >Reporter: Ariel Weisberg >Assignee: Ariel Weisberg >Priority: Normal > Fix For: 4.2 > > > Add some safety and a new convenience method to make clear that getting an > int can be done using a checked method. No need to import a separate class. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18110) Streaming fails during multiple concurrent host replacements
[ https://issues.apache.org/jira/browse/CASSANDRA-18110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654633#comment-17654633 ] Abe Ratnofsky commented on CASSANDRA-18110: --- For #3 - wouldn't disabling tracking mean that the virtual table isn't usable? This would put us in a situation where you can either use the vtable or stream successfully on larger clusters, which seems to defeat the purpose of the vtable. I'm also in favor of #4. I was also thinking of changing the pacing (via debounce) for individual file progress events, so a progress event does not trigger handleStreamEvent if it was called within the last X millis. > Streaming fails during multiple concurrent host replacements > > > Key: CASSANDRA-18110 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18110 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jon Meredith >Assignee: Jon Meredith >Priority: Normal > Fix For: 4.1.x > > > Running four concurrent host replacements on a 4.1.0 development cluster has > repeatably failed to complete bootstrap with all four hosts failing bootsrrap > and staying in JOINING, logging the message. > {code:java} > ERROR 2022-12-07T21:15:48,860 [main] > org.apache.cassandra.service.StorageService:2019 - Error while waiting on > bootstrap to complete. Bootstrap will have to be restarted. > {code} > Bootstrap fails as the the FileStreamTasks on the streaming followers > encounter an EOF while transmitting the files. > {code:java} > ERROR 2022-12-07T15:49:39,164 [NettyStreaming-Outbound-/1.2.3.4.7000:2] > org.apache.cassandra.streaming.StreamSession:718 - [Stream > #8d313690-7674-11ed-813f-95c261b64a82] Streaming error occurred on session > with peer 1.2.3.4:7000 through 1.2.3.4:40292 > org.apache.cassandra.net.AsyncChannelOutputPlus$FlushException: The channel > this output stream was writing to has been closed >at > org.apache.cassandra.net.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:158) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.waitForSpace(AsyncChannelOutputPlus.java:140) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.beginFlush(AsyncChannelOutputPlus.java:97) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.lambda$writeToChannel$0(AsyncStreamingOutputPlus.java:124) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.lambda$write$0(CassandraCompressedStreamWriter.java:89) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.writeToChannel(AsyncStreamingOutputPlus.java:120) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.write(CassandraCompressedStreamWriter.java:88) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:177) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage.serialize(OutgoingStreamMessage.java:87) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:45) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:34) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:39) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:311) > [cassandra.jar] >at > org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96) > [cassandra.jar] >at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61) > [cassandra.jar] >at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71) > [cassandra.jar] >at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > [?:?] >at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > [?:?] >at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > [netty-all-4.1.58.Final.jar:4.1.58.Final] >at java.lang.Thread.run(Thread.java:829) [?:?] >Suppressed: java.nio.channels.ClosedChannelException >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.doFlush(AsyncStreamingOutputPlus.java
[cassandra] branch trunk updated: Use checked casts when reading vints as ints
This is an automated email from the ASF dual-hosted git repository. aweisberg 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 d7c5c547f0 Use checked casts when reading vints as ints d7c5c547f0 is described below commit d7c5c547f025301780658b37e6e8a591bc4a5b36 Author: Ariel Weisberg AuthorDate: Tue Dec 6 15:20:14 2022 -0500 Use checked casts when reading vints as ints patch by Ariel Weisberg; reviewed by David Capwell and Caleb Rackliffe for CASSANDRA-18099 --- CHANGES.txt| 1 + src/java/org/apache/cassandra/batchlog/Batch.java | 10 +- .../cassandra/cql3/functions/types/TypeCodec.java | 20 +--- .../cql3/selection/AbstractFunctionSelector.java | 12 +- .../cassandra/cql3/selection/FieldSelector.java| 4 +- .../cassandra/cql3/selection/ListSelector.java | 4 +- .../cassandra/cql3/selection/MapSelector.java | 4 +- .../cassandra/cql3/selection/SetSelector.java | 4 +- .../cassandra/cql3/selection/TupleSelector.java| 4 +- .../cassandra/cql3/selection/UserTypeSelector.java | 4 +- src/java/org/apache/cassandra/db/Columns.java | 26 ++--- src/java/org/apache/cassandra/db/Mutation.java | 8 +- src/java/org/apache/cassandra/db/ReadCommand.java | 4 +- .../org/apache/cassandra/db/RowIndexEntry.java | 38 +++--- .../apache/cassandra/db/SerializationHeader.java | 18 +-- src/java/org/apache/cassandra/db/Slices.java | 4 +- .../db/aggregation/AggregationSpecification.java | 8 +- .../db/columniterator/AbstractSSTableIterator.java | 11 +- .../db/filter/ClusteringIndexNamesFilter.java | 4 +- .../apache/cassandra/db/filter/ColumnFilter.java | 4 +- .../org/apache/cassandra/db/filter/DataLimits.java | 28 ++--- .../org/apache/cassandra/db/filter/RowFilter.java | 4 +- .../apache/cassandra/db/marshal/AbstractType.java | 2 +- .../apache/cassandra/db/marshal/ValueAccessor.java | 2 +- .../cassandra/db/partitions/PartitionUpdate.java | 7 +- .../apache/cassandra/db/rows/EncodingStats.java| 8 +- .../db/rows/UnfilteredRowIteratorSerializer.java | 4 +- .../cassandra/db/rows/UnfilteredSerializer.java| 14 +-- .../cassandra/db/streaming/ComponentManifest.java | 15 +-- .../cassandra/exceptions/RequestFailureReason.java | 6 +- src/java/org/apache/cassandra/hints/Hint.java | 15 ++- .../org/apache/cassandra/hints/HintMessage.java| 4 +- .../org/apache/cassandra/hints/HintsWriter.java| 4 + .../org/apache/cassandra/io/sstable/IndexInfo.java | 12 +- .../io/sstable/SSTableIdentityIterator.java| 11 +- .../apache/cassandra/io/util/DataInputPlus.java| 33 ++ .../apache/cassandra/io/util/DataOutputPlus.java | 28 - .../cassandra/io/util/RebufferingInputStream.java | 20 +++- .../cassandra/net/CustomParamsSerializer.java | 4 +- .../org/apache/cassandra/net/ForwardingInfo.java | 19 ++- src/java/org/apache/cassandra/net/Message.java | 44 +++ .../cassandra/serializers/DurationSerializer.java | 10 +- .../cassandra/service/pager/PagingState.java | 9 +- .../service/paxos/PaxosRepairHistory.java | 4 +- .../org/apache/cassandra/utils/ByteArrayUtil.java | 4 +- .../org/apache/cassandra/utils/ByteBufferUtil.java | 6 +- .../cassandra/utils/CollectionSerializer.java | 10 +- .../apache/cassandra/utils/vint/VIntCoding.java| 114 +- .../org/apache/cassandra/net/MessageGenerator.java | 4 +- .../cassandra/distributed/impl/Instance.java | 2 +- .../cassandra/test/microbench/VIntCodingBench.java | 23 +--- .../cassandra/simulator/debug/Reconcile.java | 22 ++-- .../apache/cassandra/simulator/debug/Record.java | 59 +- .../cql3/validation/operations/CreateTest.java | 36 ++ .../org/apache/cassandra/db/RowIndexEntryTest.java | 42 +++ .../db/streaming/ComponentManifestTest.java| 48 .../cassandra/hints/ChecksummedDataInputTest.java | 24 ++-- .../apache/cassandra/hints/DTestSerializer.java| 2 +- .../io/util/BufferedDataOutputStreamTest.java | 25 ++-- .../cassandra/io/util/NIODataInputStreamTest.java | 20 ++-- .../unit/org/apache/cassandra/net/FramingTest.java | 4 +- .../cassandra/utils/vint/VIntCodingTest.java | 130 +++-- 62 files changed, 605 insertions(+), 469 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index fd5f31153c..19b67aa118 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.2 + * Use checked casts when reading vints as ints (CASSANDRA-18099) * Add Mutation Serialization Caching (CASSANDRA-17998) * Only reload compaction strategies if disk boundaries change (CASSANDRA-17874) * CEP-10: Simulator Java11 Support (CASSANDRA-17178) diff --git a/src/java/org/apache/cassandra/batchlog/Batch.j
[jira] [Commented] (CASSANDRA-18110) Streaming fails during multiple concurrent host replacements
[ https://issues.apache.org/jira/browse/CASSANDRA-18110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654630#comment-17654630 ] David Capwell commented on CASSANDRA-18110: --- [~jonmeredith] if you don't mind I can take this as you found the issue was caused by the vtable I worked on. Looking at the code / stack trace the issue looks to be in org.apache.cassandra.streaming.StreamingState#handleStreamEvent the line {code} sessions = Sessions.create(streamProgress.values()); {code} The single usage of this field is org.apache.cassandra.db.virtual.StreamingVirtualTable#updateDataSet, which is only needed when the vtable is requested... There are a few options I think we can take (can take multiple) 1) don't be eager and have org.apache.cassandra.streaming.StreamingState#sessions build on-demand from org.apache.cassandra.streaming.StreamingState#streamProgress 2) org.apache.cassandra.streaming.StreamingState#onSuccess and org.apache.cassandra.streaming.StreamingState#onFailure could build the sessions eagerly (like it does today), but push this logic to another thread 3) feature flag to allow tracking to be disabled (why I didn't do this in the first place) 4) Sessions is just received/sent so why do we need to compute using 100% of events? can we not just use a counter? Given that building 1 session can take 15m, then #1 just breaks the vtable... for this reason I am in favor of #3 and #4... ill look closer to see if I can write #4 and see if there are any tradeoffs I don't see yet > Streaming fails during multiple concurrent host replacements > > > Key: CASSANDRA-18110 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18110 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jon Meredith >Assignee: Jon Meredith >Priority: Normal > Fix For: 4.1.x > > > Running four concurrent host replacements on a 4.1.0 development cluster has > repeatably failed to complete bootstrap with all four hosts failing bootsrrap > and staying in JOINING, logging the message. > {code:java} > ERROR 2022-12-07T21:15:48,860 [main] > org.apache.cassandra.service.StorageService:2019 - Error while waiting on > bootstrap to complete. Bootstrap will have to be restarted. > {code} > Bootstrap fails as the the FileStreamTasks on the streaming followers > encounter an EOF while transmitting the files. > {code:java} > ERROR 2022-12-07T15:49:39,164 [NettyStreaming-Outbound-/1.2.3.4.7000:2] > org.apache.cassandra.streaming.StreamSession:718 - [Stream > #8d313690-7674-11ed-813f-95c261b64a82] Streaming error occurred on session > with peer 1.2.3.4:7000 through 1.2.3.4:40292 > org.apache.cassandra.net.AsyncChannelOutputPlus$FlushException: The channel > this output stream was writing to has been closed >at > org.apache.cassandra.net.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:158) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.waitForSpace(AsyncChannelOutputPlus.java:140) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncChannelOutputPlus.beginFlush(AsyncChannelOutputPlus.java:97) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.lambda$writeToChannel$0(AsyncStreamingOutputPlus.java:124) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.lambda$write$0(CassandraCompressedStreamWriter.java:89) > ~[cassandra.jar] >at > org.apache.cassandra.net.AsyncStreamingOutputPlus.writeToChannel(AsyncStreamingOutputPlus.java:120) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraCompressedStreamWriter.write(CassandraCompressedStreamWriter.java:88) > ~[cassandra.jar] >at > org.apache.cassandra.db.streaming.CassandraOutgoingFile.write(CassandraOutgoingFile.java:177) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage.serialize(OutgoingStreamMessage.java:87) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:45) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.OutgoingStreamMessage$1.serialize(OutgoingStreamMessage.java:34) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:39) > ~[cassandra.jar] >at > org.apache.cassandra.streaming.async.StreamingMultiplexedChannel$FileStreamTask.run(StreamingMultiplexedChannel.java:311) > [cassandra.jar] >
[jira] [Commented] (CASSANDRA-15458) CQL: Unable to escape single quote in a map attribute
[ https://issues.apache.org/jira/browse/CASSANDRA-15458?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654629#comment-17654629 ] Yaman Ziadeh commented on CASSANDRA-15458: -- Makes sense, I updated the tests to reflect current behavior and reverted pylib/cqlshlib/formatting.py changes. > CQL: Unable to escape single quote in a map attribute > --- > > Key: CASSANDRA-15458 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15458 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter, CQL/Semantics >Reporter: Abhijeet Singh >Assignee: Yaman Ziadeh >Priority: Normal > Labels: AdventCalendar2021, lhf > Attachments: cass-screen.png > > > h3. Overview > For {{text}} attributes, CQL allows escaping single quote [using additional > single > quote|http://mail-archives.apache.org/mod_mbox/cassandra-user/201108.mbox/%3c20110803152250.294...@gmx.net%3E]. > But applying the same syntax for a {{map}} attribute does not > work. Inconsistent behavior was observed between {{text}} datatype and > {{map}} datatype. > h3. CQL semantic proposal > Cassandra (CQL) should apply same escaping semantics on {{text}} and > text-within-map {{map}} datatypes. > h3. Reproducing the bug: > CREATE query: > {code:java} > CREATE TABLE university.test (id int, data map, PRIMARY KEY > (id));{code} > INSERT query: > {code:java} > insert into university.test (id, data) values(1, {1:'I''m newb'});{code} > On running the aforementioned INSERT query, Cassandra inserts and returns > {{\{1: 'I''m newb'}}} while the expected result is {{\{1: 'I'm newb'}}} > h3. Technical details > OS: CentOS 7 > Kernel: Linux 3.10.0-1062.9.1.el7.x86_64 > Cassandra version: 3.11.5 > Screenshot: [Output after SELECT query|https://i.stack.imgur.com/PLAan.png] > > +Additional information for newcomers:+ > The issue seems to come from the ANTLR Lexer logic for {{STRING_LITERAL}}. A > unit test should also be added in {{SelectTest}} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15458) CQL: Unable to escape single quote in a map attribute
[ https://issues.apache.org/jira/browse/CASSANDRA-15458?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654567#comment-17654567 ] Brandon Williams commented on CASSANDRA-15458: -- I agree that neither a) or b) is very desirable, so just committing the tests makes sense to me. > CQL: Unable to escape single quote in a map attribute > --- > > Key: CASSANDRA-15458 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15458 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter, CQL/Semantics >Reporter: Abhijeet Singh >Assignee: Yaman Ziadeh >Priority: Normal > Labels: AdventCalendar2021, lhf > Attachments: cass-screen.png > > > h3. Overview > For {{text}} attributes, CQL allows escaping single quote [using additional > single > quote|http://mail-archives.apache.org/mod_mbox/cassandra-user/201108.mbox/%3c20110803152250.294...@gmx.net%3E]. > But applying the same syntax for a {{map}} attribute does not > work. Inconsistent behavior was observed between {{text}} datatype and > {{map}} datatype. > h3. CQL semantic proposal > Cassandra (CQL) should apply same escaping semantics on {{text}} and > text-within-map {{map}} datatypes. > h3. Reproducing the bug: > CREATE query: > {code:java} > CREATE TABLE university.test (id int, data map, PRIMARY KEY > (id));{code} > INSERT query: > {code:java} > insert into university.test (id, data) values(1, {1:'I''m newb'});{code} > On running the aforementioned INSERT query, Cassandra inserts and returns > {{\{1: 'I''m newb'}}} while the expected result is {{\{1: 'I'm newb'}}} > h3. Technical details > OS: CentOS 7 > Kernel: Linux 3.10.0-1062.9.1.el7.x86_64 > Cassandra version: 3.11.5 > Screenshot: [Output after SELECT query|https://i.stack.imgur.com/PLAan.png] > > +Additional information for newcomers:+ > The issue seems to come from the ANTLR Lexer logic for {{STRING_LITERAL}}. A > unit test should also be added in {{SelectTest}} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15458) CQL: Unable to escape single quote in a map attribute
[ https://issues.apache.org/jira/browse/CASSANDRA-15458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paulo Motta updated CASSANDRA-15458: Status: Needs Committer (was: Patch Available) > CQL: Unable to escape single quote in a map attribute > --- > > Key: CASSANDRA-15458 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15458 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter, CQL/Semantics >Reporter: Abhijeet Singh >Assignee: Yaman Ziadeh >Priority: Normal > Labels: AdventCalendar2021, lhf > Attachments: cass-screen.png > > > h3. Overview > For {{text}} attributes, CQL allows escaping single quote [using additional > single > quote|http://mail-archives.apache.org/mod_mbox/cassandra-user/201108.mbox/%3c20110803152250.294...@gmx.net%3E]. > But applying the same syntax for a {{map}} attribute does not > work. Inconsistent behavior was observed between {{text}} datatype and > {{map}} datatype. > h3. CQL semantic proposal > Cassandra (CQL) should apply same escaping semantics on {{text}} and > text-within-map {{map}} datatypes. > h3. Reproducing the bug: > CREATE query: > {code:java} > CREATE TABLE university.test (id int, data map, PRIMARY KEY > (id));{code} > INSERT query: > {code:java} > insert into university.test (id, data) values(1, {1:'I''m newb'});{code} > On running the aforementioned INSERT query, Cassandra inserts and returns > {{\{1: 'I''m newb'}}} while the expected result is {{\{1: 'I'm newb'}}} > h3. Technical details > OS: CentOS 7 > Kernel: Linux 3.10.0-1062.9.1.el7.x86_64 > Cassandra version: 3.11.5 > Screenshot: [Output after SELECT query|https://i.stack.imgur.com/PLAan.png] > > +Additional information for newcomers:+ > The issue seems to come from the ANTLR Lexer logic for {{STRING_LITERAL}}. A > unit test should also be added in {{SelectTest}} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15458) CQL: Unable to escape single quote in a map attribute
[ https://issues.apache.org/jira/browse/CASSANDRA-15458?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654556#comment-17654556 ] Paulo Motta commented on CASSANDRA-15458: - Thanks for looking into this [~yziadeh]. It doesn't seem like this is a problem at the parser level, as shown by the server unit tests, but just a cqlsh display setting. When reviewing this I realized that the example shown in the ticket description might not be incorrect: {noformat} SELECT data from test.t1; data -- I'm newb SELECT data from test.t2; data -- {1: 'I''m newb'} {noformat} If we were to standardize quoting across simple text types vs complex types with text on cqlsh, we would have the following options: a) Quote simple text fields: {noformat} SELECT data from test.t1; data -- 'I''m newb' {noformat} b) Unquote complex types. {noformat} SELECT data from test.t2; data -- {1: I'm newb} {noformat} The problem with a) is that this would change the display of simple text fields on CQLSH, what could cause compatibility problems. The problem with b) is that it makes the presentation more ambiguous, for instance in the example above we don't know if the key '1' on }1: I'm newb{ is a text or number without looking at the schema, while }1: 'I''m newb'{ makes it immediately visible that the key '1' is a number. I belive this is working as intended, despite the confusion. I propose we commit just the tests without changing the behavior. What do you think [~stefan.miklosovic] [~brandon.williams] [~e.dimitrova]? > CQL: Unable to escape single quote in a map attribute > --- > > Key: CASSANDRA-15458 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15458 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter, CQL/Semantics >Reporter: Abhijeet Singh >Assignee: Yaman Ziadeh >Priority: Normal > Labels: AdventCalendar2021, lhf > Attachments: cass-screen.png > > > h3. Overview > For {{text}} attributes, CQL allows escaping single quote [using additional > single > quote|http://mail-archives.apache.org/mod_mbox/cassandra-user/201108.mbox/%3c20110803152250.294...@gmx.net%3E]. > But applying the same syntax for a {{map}} attribute does not > work. Inconsistent behavior was observed between {{text}} datatype and > {{map}} datatype. > h3. CQL semantic proposal > Cassandra (CQL) should apply same escaping semantics on {{text}} and > text-within-map {{map}} datatypes. > h3. Reproducing the bug: > CREATE query: > {code:java} > CREATE TABLE university.test (id int, data map, PRIMARY KEY > (id));{code} > INSERT query: > {code:java} > insert into university.test (id, data) values(1, {1:'I''m newb'});{code} > On running the aforementioned INSERT query, Cassandra inserts and returns > {{\{1: 'I''m newb'}}} while the expected result is {{\{1: 'I'm newb'}}} > h3. Technical details > OS: CentOS 7 > Kernel: Linux 3.10.0-1062.9.1.el7.x86_64 > Cassandra version: 3.11.5 > Screenshot: [Output after SELECT query|https://i.stack.imgur.com/PLAan.png] > > +Additional information for newcomers:+ > The issue seems to come from the ANTLR Lexer logic for {{STRING_LITERAL}}. A > unit test should also be added in {{SelectTest}} -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra-accord] branch trunk updated: Faster SimpleProgressLog and BurnTest (#16)
This is an automated email from the ASF dual-hosted git repository. benedict pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra-accord.git The following commit(s) were added to refs/heads/trunk by this push: new bb070c6 Faster SimpleProgressLog and BurnTest (#16) bb070c6 is described below commit bb070c6b47616e47d6370c41438715634bcd8b48 Author: Benedict Elliott Smith AuthorDate: Wed Jan 4 16:00:30 2023 + Faster SimpleProgressLog and BurnTest (#16) Improve the SimpleProgressLog to only perform work as necessary, and make some other minor improvements to performance to improve burn test throughput. Co-authored-by: Aleksey Yeschenko --- .../main/java/accord/coordinate/CheckShards.java |4 +- .../java/accord/coordinate/ReadCoordinator.java|8 +- .../coordinate/tracking/AbstractTracker.java |2 +- .../accord/coordinate/tracking/ReadTracker.java| 11 +- .../java/accord/impl/InMemoryCommandStore.java |4 +- .../main/java/accord/impl/SimpleProgressLog.java | 1072 ++-- .../java/accord/impl/SizeOfIntersectionSorter.java |2 +- .../src/main/java/accord/local/Command.java|2 +- .../src/main/java/accord/local/CommandStores.java |4 +- .../main/java/accord/messages/BeginRecovery.java | 12 +- .../src/main/java/accord/messages/Defer.java |5 +- .../main/java/accord/messages/InformDurable.java |2 + .../main/java/accord/primitives/AbstractKeys.java | 10 +- .../java/accord/primitives/AbstractRanges.java |2 +- .../src/main/java/accord/primitives/Deps.java | 74 +- .../src/main/java/accord/primitives/Routables.java |8 +- .../src/main/java/accord/primitives/Txn.java |2 +- .../src/main/java/accord/primitives/Writes.java|2 +- .../src/main/java/accord/topology/Topologies.java |2 +- .../src/main/java/accord/topology/Topology.java| 126 ++- .../main/java/accord/topology/TopologyManager.java | 14 +- .../src/main/java/accord/utils/ArrayBuffers.java | 14 +- .../main/java/accord/utils/IndexedBiFunction.java |4 +- .../main/java/accord/utils/IndexedConsumer.java|4 +- .../src/main/java/accord/utils/IndexedFold.java|3 +- .../accord/utils/IndexedFoldIntersectToLong.java |4 +- .../main/java/accord/utils/IndexedFoldToLong.java |4 +- .../main/java/accord/utils/IndexedFunction.java|4 +- .../main/java/accord/utils/IndexedIntFunction.java |4 +- .../main/java/accord/utils/IndexedPredicate.java |4 +- .../java/accord/utils/IndexedRangeFoldToLong.java |2 +- .../java/accord/utils/IndexedRangeTriConsumer.java |6 + .../main/java/accord/utils/IndexedTriConsumer.java |7 + .../main/java/accord/utils/IndexedTriFunction.java |4 +- .../java/accord/utils/IntrusiveLinkedList.java | 109 ++ ...oldToLong.java => IntrusiveLinkedListNode.java} | 24 +- .../src/main/java/accord/utils/SortedArrays.java |2 +- accord-core/src/test/java/accord/KeysTest.java | 27 +- .../src/test/java/accord/burn/BurnTest.java|9 +- .../coordinate/tracking/TrackerReconciler.java |2 +- .../src/test/java/accord/impl/basic/Cluster.java | 10 +- 41 files changed, 920 insertions(+), 695 deletions(-) diff --git a/accord-core/src/main/java/accord/coordinate/CheckShards.java b/accord-core/src/main/java/accord/coordinate/CheckShards.java index 965e6a9..dd91993 100644 --- a/accord-core/src/main/java/accord/coordinate/CheckShards.java +++ b/accord-core/src/main/java/accord/coordinate/CheckShards.java @@ -59,7 +59,9 @@ public abstract class CheckShards extends ReadCoordinator @Override protected Action process(Id from, CheckStatusReply reply) { -debug.put(from, reply); +if (debug != null) +debug.put(from, reply); + if (reply.isOk()) { CheckStatusOk ok = (CheckStatusOk) reply; diff --git a/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java b/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java index 4154b21..3c9ccbf 100644 --- a/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java +++ b/accord-core/src/main/java/accord/coordinate/ReadCoordinator.java @@ -16,6 +16,8 @@ import static com.google.common.collect.Sets.newHashSetWithExpectedSize; abstract class ReadCoordinator extends ReadTracker implements Callback { +private static final boolean DEBUG = false; + protected enum Action { /** @@ -53,7 +55,7 @@ abstract class ReadCoordinator extends Read final TxnId txnId; private boolean isDone; private Throwable failure; -Map debug = new HashMap<>(); +Map debug = DEBUG ? new HashMap<>() : null; ReadCoordinator(Node node, Topologies topologies, TxnId txnId) { @@ -69,7 +71,9 @@ abstract class ReadCoordinator extends Read @Override public void onSuccess(
[jira] [Created] (CASSANDRA-18135) TBD
Aleksey Yeschenko created CASSANDRA-18135: - Summary: TBD Key: CASSANDRA-18135 URL: https://issues.apache.org/jira/browse/CASSANDRA-18135 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko TBD -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-14013) Data loss in snapshots keyspace after service restart
[ https://issues.apache.org/jira/browse/CASSANDRA-14013?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654500#comment-17654500 ] Stefan Miklosovic commented on CASSANDRA-14013: --- [~paulo] do you plan to take a look at this again? I feel we are super-close to finish line here. > Data loss in snapshots keyspace after service restart > - > > Key: CASSANDRA-14013 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14013 > Project: Cassandra > Issue Type: Bug > Components: Legacy/Core, Local/Snapshots >Reporter: Gregor Uhlenheuer >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 4.0.x, 4.1.x, 4.x > > Time Spent: 10m > Remaining Estimate: 0h > > I am posting this bug in hope to discover the stupid mistake I am doing > because I can't imagine a reasonable answer for the behavior I see right now > :-) > In short words, I do observe data loss in a keyspace called *snapshots* after > restarting the Cassandra service. Say I do have 1000 records in a table > called *snapshots.test_idx* then after restart the table has less entries or > is even empty. > My kind of "mysterious" observation is that it happens only in a keyspace > called *snapshots*... > h3. Steps to reproduce > These steps to reproduce show the described behavior in "most" attempts (not > every single time though). > {code} > # create keyspace > CREATE KEYSPACE snapshots WITH replication = {'class': 'SimpleStrategy', > 'replication_factor': 1}; > # create table > CREATE TABLE snapshots.test_idx (key text, seqno bigint, primary key(key)); > # insert some test data > INSERT INTO snapshots.test_idx (key,seqno) values ('key1', 1); > ... > INSERT INTO snapshots.test_idx (key,seqno) values ('key1000', 1000); > # count entries > SELECT count(*) FROM snapshots.test_idx; > 1000 > # restart service > kill > cassandra -f > # count entries > SELECT count(*) FROM snapshots.test_idx; > 0 > {code} > I hope someone can point me to the obvious mistake I am doing :-) > This happened to me using both Cassandra 3.9 and 3.11.0 -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-16325) Update streaming metrics incrementally
[ https://issues.apache.org/jira/browse/CASSANDRA-16325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-16325: -- Status: Changes Suggested (was: Review In Progress) > Update streaming metrics incrementally > -- > > Key: CASSANDRA-16325 > URL: https://issues.apache.org/jira/browse/CASSANDRA-16325 > Project: Cassandra > Issue Type: Improvement > Components: Observability/Metrics >Reporter: Paulo Motta >Assignee: Isaac Reath >Priority: Normal > Labels: lhf > Time Spent: 0.5h > Remaining Estimate: 0h > > Currently the inbound and outbound streamed bytes metrics are incremented > after each file is streamed, what doesn't represent the current number of > bytes streamed since it can take a long time for a large file to be streamed. > We should update the metric incrementally as data is streamed. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-16325) Update streaming metrics incrementally
[ https://issues.apache.org/jira/browse/CASSANDRA-16325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Miklosovic updated CASSANDRA-16325: -- Reviewers: Paulo Motta, Stefan Miklosovic (was: Paulo Motta) Status: Review In Progress (was: Needs Committer) > Update streaming metrics incrementally > -- > > Key: CASSANDRA-16325 > URL: https://issues.apache.org/jira/browse/CASSANDRA-16325 > Project: Cassandra > Issue Type: Improvement > Components: Observability/Metrics >Reporter: Paulo Motta >Assignee: Isaac Reath >Priority: Normal > Labels: lhf > Time Spent: 0.5h > Remaining Estimate: 0h > > Currently the inbound and outbound streamed bytes metrics are incremented > after each file is streamed, what doesn't represent the current number of > bytes streamed since it can take a long time for a large file to be streamed. > We should update the metric incrementally as data is streamed. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-16325) Update streaming metrics incrementally
[ https://issues.apache.org/jira/browse/CASSANDRA-16325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654491#comment-17654491 ] Stefan Miklosovic commented on CASSANDRA-16325: --- Ive put some comments in. > Update streaming metrics incrementally > -- > > Key: CASSANDRA-16325 > URL: https://issues.apache.org/jira/browse/CASSANDRA-16325 > Project: Cassandra > Issue Type: Improvement > Components: Observability/Metrics >Reporter: Paulo Motta >Assignee: Isaac Reath >Priority: Normal > Labels: lhf > Time Spent: 20m > Remaining Estimate: 0h > > Currently the inbound and outbound streamed bytes metrics are incremented > after each file is streamed, what doesn't represent the current number of > bytes streamed since it can take a long time for a large file to be streamed. > We should update the metric incrementally as data is streamed. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-18134) Improve handling of min/max clustering in sstable
Jacek Lewandowski created CASSANDRA-18134: - Summary: Improve handling of min/max clustering in sstable Key: CASSANDRA-18134 URL: https://issues.apache.org/jira/browse/CASSANDRA-18134 Project: Cassandra Issue Type: Improvement Components: Local/SSTable Reporter: Jacek Lewandowski Assignee: Jacek Lewandowski (details will come later) -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-18018) List command output not correct for super user, after grant command
[ https://issues.apache.org/jira/browse/CASSANDRA-18018?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17654449#comment-17654449 ] Sam Tunnicliffe commented on CASSANDRA-18018: - Resources are organised hierarchically, with each \{IResource} implementation having a root resource which "contains" all other resources of that type. So you don't need to enumerate every individual resource, you can just show the applicable permissions for each of the root resources. So permissions for any superuser would look like: {code} cassandra@cqlsh> LIST ALL PERMISSIONS OF superuser_a; role | username | resource | permission -+-+-+ superuser_a | superuser_a | | CREATE superuser_a | superuser_a | | ALTER superuser_a | superuser_a | | DROP superuser_a | superuser_a | | SELECT superuser_a | superuser_a | | MODIFY superuser_a | superuser_a | | AUTHORIZE superuser_a | superuser_a | | CREATE superuser_a | superuser_a | | ALTER superuser_a | superuser_a | | DROP superuser_a | superuser_a | | AUTHORIZE superuser_a | superuser_a | | EXECUTE superuser_a | superuser_a | | SELECT superuser_a | superuser_a | | MODIFY superuser_a | superuser_a | | AUTHORIZE superuser_a | superuser_a | | DESCRIBE superuser_a | superuser_a | | EXECUTE superuser_a | superuser_a | | CREATE superuser_a | superuser_a | | ALTER superuser_a | superuser_a | | DROP superuser_a | superuser_a | | AUTHORIZE superuser_a | superuser_a | | DESCRIBE (21 rows) {code} > List command output not correct for super user, after grant command > --- > > Key: CASSANDRA-18018 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18018 > Project: Cassandra > Issue Type: Bug > Components: Feature/Authorization >Reporter: Shailaja Koppu >Assignee: Maxim Chanturiay >Priority: Normal > Labels: lhf > > Running local Cassandra with below config: > {noformat} > authenticator: PasswordAuthenticator > authorizer: CassandraAuthorizer > role_manager: CassandraRoleManager > network_authorizer: CassandraNetworkAuthorizer{noformat} > Created a super user and then ran *Grant select* command on a keyspace. > {noformat} > shaadmin1@cqlsh> CREATE USER 'shaadmin1c1' WITH PASSWORD 'shaadmin1c1' > SUPERUSER; > shaadmin1@cqlsh:system_auth> grant select on testk1.t1 to shaadmin1c1; > shaadmin1@cqlsh:system_auth> alter role shaadmin1c1 with access to all > datacenters; > {noformat} > > After this, list permissions command showing only select permission for that > role on the resource. > {noformat} > shaadmin1c1@cqlsh> list all permissions of shaadmin1c1; > role | username | resource | permission > +--- > shaadmin1c1 | shaadmin1c1 | | SELECT > {noformat} > > Row in role_permissions table: > {noformat} > role | resource | permissions > -- > shaadmin1c1 | data/testk1/t1 | {'SELECT'}{noformat} > But insert command by that role on the resource is successful because role is > a super user > {noformat} > shaadmin1c1@cqlsh> insert into testk1.t1 (c1, c2) values ('a', 1); > shaadmin1c1@cqlsh> select * from testk1.t1 ; > c1 | c2 > ---+--- > a | 1 > (1 rows) > {noformat} > > The problem is, output of list permissions command, which indicates only > select permission on the resource, is misleading. I think list command need > to be fixed to show all permissions super user has on the resource. Also > grant command for a super user can be either a no-op or throw error, because > the role already have requested permissions. > > Documentation also misleading: > {quote}True automatically grants AUTHORIZE, CREATE and DROP permission on ALL > ROLES. > Superusers can only manage roles by default. To manage other resources, > {color:#ff}you must grant the permission set to that resource. ** > {color}For example, to allow access management for all keyspaces: {{{}GRANT > ALL PERMISSIONS ON ALL KEYSPACES TO }}\{{{}{*}role_name{*}{}}}. > {quote} > > > -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-17056) Pluggable SSTable format (SSTable format API)
[ https://issues.apache.org/jira/browse/CASSANDRA-17056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jacek Lewandowski updated CASSANDRA-17056: -- Test and Documentation Plan: Run all regressions Check API doc comments Check documentation Check NEWS.txt Status: Patch Available (was: In Progress) PR: https://github.com/apache/cassandra/pull/2064 > Pluggable SSTable format (SSTable format API) > - > > Key: CASSANDRA-17056 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17056 > Project: Cassandra > Issue Type: New Feature > Components: Local/SSTable >Reporter: Jacek Lewandowski >Assignee: Jacek Lewandowski >Priority: Normal > Time Spent: 10m > Remaining Estimate: 0h > > https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-17%3A+SSTable+format+API -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-18133) In-tree build scripts
[ https://issues.apache.org/jira/browse/CASSANDRA-18133?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Semb Wever updated CASSANDRA-18133: --- Change Category: Quality Assurance Complexity: Normal Fix Version/s: 4.x Reviewers: Ekaterina Dimitrova Status: Open (was: Triage Needed) > In-tree build scripts > - > > Key: CASSANDRA-18133 > URL: https://issues.apache.org/jira/browse/CASSANDRA-18133 > Project: Cassandra > Issue Type: Task > Components: CI >Reporter: Michael Semb Wever >Assignee: Michael Semb Wever >Priority: Normal > Fix For: 4.x > > > To simplify the introduction of JDK17 (and removal of JDK8), bring the > artifact/deb/rpm build scripts (and associated docker images) from > cassandra-builds repo to the .build directory. > The declarative Jenkinsfile can then directly declare the artifacts jobs in > its pipeline. And the packaging jobs can be separated and run in parallel. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-18133) In-tree build scripts
Michael Semb Wever created CASSANDRA-18133: -- Summary: In-tree build scripts Key: CASSANDRA-18133 URL: https://issues.apache.org/jira/browse/CASSANDRA-18133 Project: Cassandra Issue Type: Task Components: CI Reporter: Michael Semb Wever Assignee: Michael Semb Wever To simplify the introduction of JDK17 (and removal of JDK8), bring the artifact/deb/rpm build scripts (and associated docker images) from cassandra-builds repo to the .build directory. The declarative Jenkinsfile can then directly declare the artifacts jobs in its pipeline. And the packaging jobs can be separated and run in parallel. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org