[jira] [Commented] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2

2018-10-23 Thread Xiaodong Xie (JIRA)


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

Xiaodong Xie commented on CASSANDRA-1:
--

Yes, I agree that this is a duplicate of 10880. Please close this one. Thanks. 

> Got NPE when querying Cassandra 3.11.2
> --
>
> Key: CASSANDRA-1
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
> Environment: Ubuntu 14.04, JDK 1.8.0_171. 
> Cassandra 3.11.2
>Reporter: Xiaodong Xie
>Assignee: Xiaodong Xie
>Priority: Blocker
>  Labels: pull-request-available
> Fix For: 3.11.x
>
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2
> After upgrading, we immediately got exceptions in Cassandra like this one: 
>  
> {code}
> ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 
> QueryMessage.java:129 - Unexpected error during query
> java.lang.NullPointerException: null
> at 
> org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> 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_171]
> at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
> [apache-cassandra-3.11.2.jar:3.11.2]
> at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
> {code}
>  
> The table schema is like:
> {code}
> CREATE TABLE example.example_table (
>  id bigint,
>  hash text,
>  json text,
>  PRIMARY KEY (id, hash)
> ) WITH COMPACT STORAGE
> {code}
>  
> The query is something like:
> {code}
> "select * from example.example_table;" // (We do know this is bad practise, 
> and we are trying to fix that right now)
> {code}
> with fetch-size as 200, using DataStax Java driver. 
> This table contains about 20k rows. 
>  
> Actually, the fix is quite simple, 
>  
> {code}
> --- a/src/java/org/apache/cassandra/service/pager/PagingState.java
> +++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
> @@ -46,7 +46,7 @@ public class PagingState
> public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, 
> int remainingInPartition)
>  {
> - this.partitionKey = partitionKey;
> + this.partitionKey = 

[jira] [Commented] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2

2018-06-26 Thread Xiaodong Xie (JIRA)


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

Xiaodong Xie commented on CASSANDRA-1:
--

Ah, thanks for the explanation, [~eperott], what you saw in your test actually 
happened during our upgrade. Without the patch, we were seeing tens to hundreds 
of NPEs per seconds in the Cassandra cluster; while after this patch, the 
Cassandra cluster was quiet, but the app started to throw the exception you 
mentioned (far less often, probably several per minute in average, which was 
tolerable in our use case). 

I thought we might encounter this bug in the driver: 
[https://datastax-oss.atlassian.net/browse/JAVA-1740,] but the exception still 
happened after we bumped the driver version to 3.5.0. 

Thanks again for your test and explanation, [~eperott]. (y)

> Got NPE when querying Cassandra 3.11.2
> --
>
> Key: CASSANDRA-1
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
> Environment: Ubuntu 14.04, JDK 1.8.0_171. 
> Cassandra 3.11.2
>Reporter: Xiaodong Xie
>Priority: Blocker
>
> We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2
> After upgrading, we immediately got exceptions in Cassandra like this one: 
>  
> {code}
> ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 
> QueryMessage.java:129 - Unexpected error during query
> java.lang.NullPointerException: null
> at 
> org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> 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_171]
> at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
> [apache-cassandra-3.11.2.jar:3.11.2]
> at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
> {code}
>  
> The table schema is like:
> {code}
> CREATE TABLE example.example_table (
>  id bigint,
>  hash text,
>  json text,
>  PRIMARY KEY (id, hash)
> ) WITH COMPACT STORAGE
> {code}
>  
> The query is something like:
> {code}
> "select * from example.example_table;" // (We do know this is bad practise, 
> and we are trying to fix that right now)
> {code}
> with fetch-size as 200, using DataStax Java driver. 
> This table contains about 20k rows. 

[jira] [Commented] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2

2018-06-25 Thread Xiaodong Xie (JIRA)


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

Xiaodong Xie commented on CASSANDRA-1:
--

Thanks for the comments, [~eperott]. (y)

We've already successfully upgraded our cluster (all 6 nodes) to 3.11.2 (from 
2.2.6 as mentioned) with the patch. And we are running 3.11.2 plus the patch 
since then for about 1 month, everything was fine. 

 

> Got NPE when querying Cassandra 3.11.2
> --
>
> Key: CASSANDRA-1
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
> Environment: Ubuntu 14.04, JDK 1.8.0_171. 
> Cassandra 3.11.2
>Reporter: Xiaodong Xie
>Priority: Blocker
>
> We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2
> After upgrading, we immediately got exceptions in Cassandra like this one: 
>  
> {code}
> ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 
> QueryMessage.java:129 - Unexpected error during query
> java.lang.NullPointerException: null
> at 
> org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> 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_171]
> at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
> [apache-cassandra-3.11.2.jar:3.11.2]
> at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
> {code}
>  
> The table schema is like:
> {code}
> CREATE TABLE example.example_table (
>  id bigint,
>  hash text,
>  json text,
>  PRIMARY KEY (id, hash)
> ) WITH COMPACT STORAGE
> {code}
>  
> The query is something like:
> {code}
> "select * from example.example_table;" // (We do know this is bad practise, 
> and we are trying to fix that right now)
> {code}
> with fetch-size as 200, using DataStax Java driver. 
> This table contains about 20k rows. 
>  
> Actually, the fix is quite simple, 
>  
> {code}
> --- a/src/java/org/apache/cassandra/service/pager/PagingState.java
> +++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
> @@ -46,7 +46,7 @@ public class PagingState
> public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, 
> int remainingInPartition)
>  {
> - this.partitionKey = partitionKey;
> + this.partitionKey = 

[jira] [Updated] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2

2018-05-16 Thread Xiaodong Xie (JIRA)

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

Xiaodong Xie updated CASSANDRA-1:
-
Priority: Blocker  (was: Major)

> Got NPE when querying Cassandra 3.11.2
> --
>
> Key: CASSANDRA-1
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
> Environment: Ubuntu 14.04, JDK 1.8.0_171. 
> Cassandra 3.11.2
>Reporter: Xiaodong Xie
>Priority: Blocker
>
> We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2
> After upgrading, we immediately got exceptions in Cassandra like this one: 
>  
> ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 
> QueryMessage.java:129 - Unexpected error during query
> java.lang.NullPointerException: null
> at 
> org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> 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_171]
> at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
> [apache-cassandra-3.11.2.jar:3.11.2]
> at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
>  
> The table schema is like:
> CREATE TABLE example.example_table (
>  id bigint,
>  hash text,
>  json text,
>  PRIMARY KEY (id, hash)
> ) WITH COMPACT STORAGE
>  
> The query is something like:
> "select * from example.example_table;" // (We do know this is bad practise, 
> and we are trying to fix that right now)
> with fetch-size as 200, using DataStax Java driver. 
> This table contains about 20k rows. 
>  
> Actually, the fix is quite simple, 
>  
> --- a/src/java/org/apache/cassandra/service/pager/PagingState.java
> +++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
> @@ -46,7 +46,7 @@ public class PagingState
> public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, 
> int remainingInPartition)
>  {
> - this.partitionKey = partitionKey;
> + this.partitionKey = partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER 
> : partitionKey;
>  this.rowMark = rowMark;
>  this.remaining = remaining;
>  this.remainingInPartition = remainingInPartition;
>  
> "partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : partitionKey;" is 
> in 2.2.6 and 2.2.8. But it was removed for some reason. 

[jira] [Commented] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2

2018-05-11 Thread Xiaodong Xie (JIRA)

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

Xiaodong Xie commented on CASSANDRA-1:
--

Here is my PR: [https://github.com/apache/cassandra/pull/225]

Could anyone please have a look? Thanks a lot. 

> Got NPE when querying Cassandra 3.11.2
> --
>
> Key: CASSANDRA-1
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1
> Project: Cassandra
>  Issue Type: Bug
>  Components: CQL
> Environment: Ubuntu 14.04, JDK 1.8.0_171. 
> Cassandra 3.11.2
>Reporter: Xiaodong Xie
>Priority: Major
>
> We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2
> After upgrading, we immediately got exceptions in Cassandra like this one: 
>  
> ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 
> QueryMessage.java:129 - Unexpected error during query
> java.lang.NullPointerException: null
> at 
> org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) 
> ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116)
>  ~[apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> 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_171]
> at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>  [apache-cassandra-3.11.2.jar:3.11.2]
> at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
> [apache-cassandra-3.11.2.jar:3.11.2]
> at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]
>  
> The table schema is like:
> CREATE TABLE example.example_table (
>  id bigint,
>  hash text,
>  json text,
>  PRIMARY KEY (id, hash)
> ) WITH COMPACT STORAGE
>  
> The query is something like:
> "select * from example.example_table;" // (We do know this is bad practise, 
> and we are trying to fix that right now)
> with fetch-size as 200, using DataStax Java driver. 
> This table contains about 20k rows. 
>  
> Actually, the fix is quite simple, 
>  
> --- a/src/java/org/apache/cassandra/service/pager/PagingState.java
> +++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
> @@ -46,7 +46,7 @@ public class PagingState
> public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, 
> int remainingInPartition)
>  {
> - this.partitionKey = partitionKey;
> + this.partitionKey = partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER 
> : partitionKey;
>  this.rowMark = rowMark;
>  this.remaining = remaining;
>  this.remainingInPartition = remainingInPartition;
>  
> "partitionKey == null 

[jira] [Created] (CASSANDRA-14444) Got NPE when querying Cassandra 3.11.2

2018-05-11 Thread Xiaodong Xie (JIRA)
Xiaodong Xie created CASSANDRA-1:


 Summary: Got NPE when querying Cassandra 3.11.2
 Key: CASSANDRA-1
 URL: https://issues.apache.org/jira/browse/CASSANDRA-1
 Project: Cassandra
  Issue Type: Bug
  Components: CQL
 Environment: Ubuntu 14.04, JDK 1.8.0_171. 

Cassandra 3.11.2
Reporter: Xiaodong Xie


We just upgraded our Cassandra cluster from 2.2.6 to 3.11.2

After upgrading, we immediately got exceptions in Cassandra like this one: 

 

ERROR [Native-Transport-Requests-1] 2018-05-11 17:10:21,994 
QueryMessage.java:129 - Unexpected error during query
java.lang.NullPointerException: null
at 
org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:248) 
~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:92)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at org.apache.cassandra.config.CFMetaData.decorateKey(CFMetaData.java:666) 
~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.service.pager.PartitionRangeQueryPager.(PartitionRangeQueryPager.java:44)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.db.PartitionRangeReadCommand.getPager(PartitionRangeReadCommand.java:268)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.cql3.statements.SelectStatement.getPager(SelectStatement.java:475)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:288)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:118)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:224)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:255) 
~[apache-cassandra-3.11.2.jar:3.11.2]
at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:240) 
~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:116)
 ~[apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:517)
 [apache-cassandra-3.11.2.jar:3.11.2]
at 
org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
 [apache-cassandra-3.11.2.jar:3.11.2]
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_171]
at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
 [apache-cassandra-3.11.2.jar:3.11.2]
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) 
[apache-cassandra-3.11.2.jar:3.11.2]
at java.lang.Thread.run(Thread.java:748) [na:1.8.0_171]

 

The table schema is like:

CREATE TABLE example.example_table (
 id bigint,
 hash text,
 json text,
 PRIMARY KEY (id, hash)
) WITH COMPACT STORAGE

 

The query is something like:

"select * from example.example_table;" // (We do know this is bad practise, and 
we are trying to fix that right now)

with fetch-size as 200, using DataStax Java driver. 

This table contains about 20k rows. 

 

Actually, the fix is quite simple, 

 

--- a/src/java/org/apache/cassandra/service/pager/PagingState.java
+++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
@@ -46,7 +46,7 @@ public class PagingState

public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, int 
remainingInPartition)
 {
- this.partitionKey = partitionKey;
+ this.partitionKey = partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : 
partitionKey;
 this.rowMark = rowMark;
 this.remaining = remaining;
 this.remainingInPartition = remainingInPartition;

 

"partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : partitionKey;" is in 
2.2.6 and 2.2.8. But it was removed for some reason. 

The interesting part is that, we have: 

public final ByteBuffer partitionKey; // Can be null for single partition 
queries.

It seems "partitionKey" could be null.

Thanks a lot. 

 

 

 



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

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