[jira] [Commented] (CASSANDRA-14087) NPE when CAS encounters empty frozen collection
[ https://issues.apache.org/jira/browse/CASSANDRA-14087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16368758#comment-16368758 ] Kurt Greaves commented on CASSANDRA-14087: -- Turns out I didn't run the unit test on trunk, just manually tested on trunk. So we're throwing an {{AssertionError}} [here|https://github.com/apache/cassandra/blob/8b3a60b9a7dbefeecc06bace617279612ec7092d/src/java/org/apache/cassandra/cql3/Terms.java#L169] because a {{Lists.Marker}} is created out of the prepared statement used in the test. The assertion kind of implies you can't use a list in the condition of a prepared statement, but this was always possible previously. It's really unclear why that assert statement is there, and removing it solves all the problems. Anyway, LMK what you think. [~blerer] will likely know more about the assert. [trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:14087-trunk] > NPE when CAS encounters empty frozen collection > --- > > Key: CASSANDRA-14087 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14087 > Project: Cassandra > Issue Type: Bug >Reporter: Jens Bannmann >Assignee: Kurt Greaves >Priority: Major > Fix For: 3.0.x, 3.11.x > > > When a compare-and-set operation specifying an equality criterion with a > non-{{null}} value encounters an empty collection ({{null}} cell), the server > throws a {{NullPointerException}} and the query fails. > This does not happen for non-frozen collections. > There's a self-contained test case at > [github|https://github.com/incub8/cassandra-npe-in-cas]. > The stack trace for 3.11.0 is: > {code} > ERROR [Native-Transport-Requests-1] 2017-11-27 12:59:26,924 > QueryMessage.java:129 - Unexpected error during query > java.lang.NullPointerException: null > at > org.apache.cassandra.cql3.ColumnCondition$CollectionBound.appliesTo(ColumnCondition.java:546) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.statements.CQL3CasRequest$ColumnsConditions.appliesTo(CQL3CasRequest.java:324) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.statements.CQL3CasRequest.appliesTo(CQL3CasRequest.java:210) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.service.StorageProxy.cas(StorageProxy.java:265) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.statements.ModificationStatement.executeWithCondition(ModificationStatement.java:441) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:416) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:217) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:248) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:233) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116) > ~[apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517) > [apache-cassandra-3.11.0.jar:3.11.0] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410) > [apache-cassandra-3.11.0.jar:3.11.0] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_151] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > [apache-cassandra-3.11.0.jar:3.11.0] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [apache-cassandra-3.11.0.jar:3.11.0] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_151] > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail:
[jira] [Commented] (CASSANDRA-11381) Node running with join_ring=false and authentication can not serve requests
[ https://issues.apache.org/jira/browse/CASSANDRA-11381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16368733#comment-16368733 ] ASF GitHub Bot commented on CASSANDRA-11381: GitHub user michaelsembwever opened a pull request: https://github.com/apache/cassandra-dtest/pull/19 New test for CASSANDRA-11381: Node running with join_ring=false and authentication can not serve requests Follow up to CASSANDRA-11381 and https://github.com/riptano/cassandra-dtest/pull/1479 I think the concerns raised previously [here|https://github.com/riptano/cassandra-dtest/pull/1479#issuecomment-315443711] have been addressed. fyi @jkni , @ptnapoleon You can merge this pull request into a Git repository by running: $ git pull https://github.com/thelastpickle/cassandra-dtest mck/master_11381 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/cassandra-dtest/pull/19.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #19 commit e6daf37605e50964fdc0a141e927b37d715b6820 Author: mckDate: 2017-05-03T08:29:10Z New test for CASSANDRA-11381: Node running with join_ring=false and authentication can not serve requests Patch by Mick Semb Wever; Reviewed by XXX for CASSANDRA-11381 > Node running with join_ring=false and authentication can not serve requests > --- > > Key: CASSANDRA-11381 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11381 > Project: Cassandra > Issue Type: Bug >Reporter: mck >Assignee: mck >Priority: Major > Fix For: 2.2.10, 3.0.14, 3.11.0, 4.0 > > > Starting up a node with {{-Dcassandra.join_ring=false}} in a cluster that has > authentication configured, eg PasswordAuthenticator, won't be able to serve > requests. This is because {{Auth.setup()}} never gets called during the > startup. > Without {{Auth.setup()}} having been called in {{StorageService}} clients > connecting to the node fail with the node throwing > {noformat} > java.lang.NullPointerException > at > org.apache.cassandra.auth.PasswordAuthenticator.authenticate(PasswordAuthenticator.java:119) > at > org.apache.cassandra.thrift.CassandraServer.login(CassandraServer.java:1471) > at > org.apache.cassandra.thrift.Cassandra$Processor$login.getResult(Cassandra.java:3505) > at > org.apache.cassandra.thrift.Cassandra$Processor$login.getResult(Cassandra.java:3489) > at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) > at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) > at com.thinkaurelius.thrift.Message.invoke(Message.java:314) > at > com.thinkaurelius.thrift.Message$Invocation.execute(Message.java:90) > at > com.thinkaurelius.thrift.TDisruptorServer$InvocationHandler.onEvent(TDisruptorServer.java:695) > at > com.thinkaurelius.thrift.TDisruptorServer$InvocationHandler.onEvent(TDisruptorServer.java:689) > at com.lmax.disruptor.WorkProcessor.run(WorkProcessor.java:112) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > {noformat} > The exception thrown from the > [code|https://github.com/apache/cassandra/blob/cassandra-2.0.16/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java#L119] > {code} > ResultMessage.Rows rows = > authenticateStatement.execute(QueryState.forInternalCalls(), new > QueryOptions(consistencyForUser(username), > >Lists.newArrayList(ByteBufferUtil.bytes(username; > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org