[ https://issues.apache.org/jira/browse/CASSANDRA-15172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16900876#comment-16900876 ]
feroz shaik commented on CASSANDRA-15172: ----------------------------------------- Full stack trace is as below: WARN [ReadStage-4] 2019-08-06 02:57:57,408 AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread Thread[ReadStage-4,5,main]: {} java.lang.NullPointerException: null at org.apache.cassandra.db.LegacyLayout$LegacyRangeTombstoneList.updateDigest(LegacyLayout.java:2433) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.LegacyLayout$LegacyUnfilteredPartition.digest(LegacyLayout.java:1479) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.rows.UnfilteredRowIterators.digest(UnfilteredRowIterators.java:182) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.partitions.UnfilteredPartitionIterators.digest(UnfilteredPartitionIterators.java:263) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.ReadResponse.makeDigest(ReadResponse.java:140) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.ReadResponse.createDigestResponse(ReadResponse.java:87) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:352) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:50) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:66) ~[apache-cassandra-3.11.4.jar:3.11.4] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_131] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) ~[apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134) [apache-cassandra-3.11.4.jar:3.11.4] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:114) [apache-cassandra-3.11.4.jar:3.11.4] at java.lang.Thread.run(Thread.java:748) [na:1.8.0_131] > AbstractLocalAwareExecutorService Exception During Upgrade to 3.11.4 > -------------------------------------------------------------------- > > Key: CASSANDRA-15172 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15172 > Project: Cassandra > Issue Type: Bug > Reporter: Shalom > Priority: Normal > > Hi All, > This is the first time I open an issue, so apologies if I'm not following the > rules properly. > > After upgrading a node from version 2.1.21 to 3.11.4, we've started seeing a > lot of AbstractLocalAwareExecutorService exceptions. This happened right > after the node successfully started up with the new 3.11.4 binaries. > INFO [main] 2019-06-05 04:41:37,730 Gossiper.java:1715 - No gossip backlog; > proceeding > INFO [main] 2019-06-05 04:41:38,036 NativeTransportService.java:70 - Netty > using native Epoll event loop > INFO [main] 2019-06-05 04:41:38,117 Server.java:155 - Using Netty Version: > [netty-buffer=netty-buffer-4.0.44.Final.452812a, > netty-codec=netty-codec-4.0.44.Final.452812a, > netty-codec-haproxy=netty-codec-haproxy-4.0.44.Final.452812a, > netty-codec-http=netty-codec-http-4.0.44.Final.452812a, > netty-codec-socks=netty-codec-socks-4.0.44.Final.452812a, > netty-common=netty-common-4.0.44.Final.452812a, > netty-handler=netty-handler-4.0.44.Final.452812a, > netty-tcnative=netty-tcnative-1.1.33.Fork26.142ecbb, > netty-transport=netty-transport-4.0.44.Final.452812a, > netty-transport-native-epoll=netty-transport-native-epoll-4.0.44.Final.452812a, > netty-transport-rxtx=netty-transport-rxtx-4.0.44.Final.452812a, > netty-transport-sctp=netty-transport-sctp-4.0.44.Final.452812a, > netty-transport-udt=netty-transport-udt-4.0.44.Final.452812a] > INFO [main] 2019-06-05 04:41:38,118 Server.java:156 - Starting listening for > CQL clients on /0.0.0.0:9042 (unencrypted)... > INFO [main] 2019-06-05 04:41:38,179 CassandraDaemon.java:556 - Not starting > RPC server as requested. Use JMX (StorageService->startRPCServer()) or > nodetool (enablethrift) to start it > INFO [Native-Transport-Requests-21] 2019-06-05 04:41:39,145 > AuthCache.java:161 - (Re)initializing PermissionsCache (validity > period/update interval/max entries) (2000/2000/1000) > INFO [OptionalTasks:1] 2019-06-05 04:41:39,729 CassandraAuthorizer.java:409 > - Converting legacy permissions data > INFO [HANDSHAKE-/10.10.10.8] 2019-06-05 04:41:39,808 > OutboundTcpConnection.java:561 - Handshaking version with /10.10.10.8 > INFO [HANDSHAKE-/10.10.10.9] 2019-06-05 04:41:39,808 > OutboundTcpConnection.java:561 - Handshaking version with /10.10.10.9 > INFO [HANDSHAKE-dc1_02/10.10.10.6] 2019-06-05 04:41:39,809 > OutboundTcpConnection.java:561 - Handshaking version with dc1_02/10.10.10.6 > WARN [ReadStage-2] 2019-06-05 04:41:39,857 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-2,5,main]: {} > java.lang.ArrayIndexOutOfBoundsException: 1 > at > org.apache.cassandra.db.AbstractBufferClusteringPrefix.get(AbstractBufferClusteringPrefix.java:55) > at > org.apache.cassandra.db.LegacyLayout$LegacyRangeTombstoneList.serializedSizeCompound(LegacyLayout.java:2545) > at > org.apache.cassandra.db.LegacyLayout$LegacyRangeTombstoneList.serializedSize(LegacyLayout.java:2522) > at > org.apache.cassandra.db.LegacyLayout.serializedSizeAsLegacyPartition(LegacyLayout.java:565) > at > org.apache.cassandra.db.ReadResponse$Serializer.serializedSize(ReadResponse.java:446) > at > org.apache.cassandra.db.ReadResponse$Serializer.serializedSize(ReadResponse.java:352) > at > org.apache.cassandra.net.MessageOut.payloadSize(MessageOut.java:171) > at > org.apache.cassandra.net.OutboundTcpConnectionPool.getConnection(OutboundTcpConnectionPool.java:77) > at > org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:802) > at > org.apache.cassandra.net.MessagingService.sendOneWay(MessagingService.java:953) > at > org.apache.cassandra.net.MessagingService.sendReply(MessagingService.java:929) > at > org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:62) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:66) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:134) > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:114) > at java.lang.Thread.run(Thread.java:745) > > > After several of the above warnings, the following warning appeared as well: > WARN [ReadStage-9] 2019-06-05 04:42:04,369 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-9,5,main]: {} > java.lang.ArrayIndexOutOfBoundsException: null > WARN [ReadStage-11] 2019-06-05 04:42:04,381 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-11,5,main]: {} > java.lang.ArrayIndexOutOfBoundsException: null > WARN [ReadStage-10] 2019-06-05 04:42:04,396 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-10,5,main]: {} > java.lang.ArrayIndexOutOfBoundsException: null > WARN [ReadStage-2] 2019-06-05 04:42:04,443 > AbstractLocalAwareExecutorService.java:167 - Uncaught exception on thread > Thread[ReadStage-2,5,main]: {} > java.lang.ArrayIndexOutOfBoundsException: null > > Then suddenly, Validation errors appeared although *no repair was running on > any of the nodes*! Checked with ps -ef command and nodetool compactionstats > on the entire cluster. > > ERROR [ValidationExecutor:2] 2019-06-05 04:42:47,979 Validator.java:268 - > Failed creating a merkle tree for [repair > #e54b4090-876d-11e9-a3f4-c33d22c45471 on ks1/table1, []], / > 10.10.10.6 (see log for details) > ERROR [ValidationExecutor:2] 2019-06-05 04:42:47,979 CassandraDaemon.java:228 > - Exception in thread Thread[ValidationExecutor:2,1,main] > java.lang.NullPointerException: null > at > org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1363) > at > org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:83) > at > org.apache.cassandra.db.compaction.CompactionManager$13.call(CompactionManager.java:977) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) > at java.lang.Thread.run(Thread.java:745) > > Following those, client requests started to fail and NTR tasks started to > pile up and get blocked and GC was impacted. > INFO [ScheduledTasks:1] 2019-06-05 04:43:11,660 StatusLogger.java:51 - > Native-Transport-Requests 128 197 594810 65 > 2725 > > FWIW, these are the warnings I found during startup: > -WARN in net.logstash.logback.encoder.LogstashEncoder@140e5a13 - Logback > version is prior to 1.2.0. Enabling backwards compatible encoding. Logback > 1.2.1 or greater is recommended. > > WARN [main] 2019-06-05 08:44:18,568 NativeLibrary.java:187 - Unable to lock > JVM memory (ENOMEM). This can result in part of the JVM being swapped out, > especially with mmapped I/O enabled. Increase RLIMIT_MEMLOCK or run Cassandra > as root. > WARN [main] 2019-06-05 08:44:18,569 StartupChecks.java:136 - jemalloc shared > library could not be preloaded to speed up memory allocations > > WARN [main] 2019-06-05 08:44:20,225 Optional.java:159 - Legacy auth tables > credentials, users, permissions in keyspace system_auth still exist and have > not been properly migrated. > WARN [MessagingService-Outgoing-dc1_03/10.10.10.4-Gossip] 2019-06-05 > 08:44:49,582 OutboundTcpConnection.java:486 - Seed gossip version is 8; will > not connect with that version > WARN [MessagingService-Outgoing-dc2_02/10.20.20.4-Gossip] 2019-06-05 > 08:44:49,620 OutboundTcpConnection.java:486 - Seed gossip version is 8; will > not connect with that version > WARN [MessagingService-Outgoing-dc2_01/10.20.20.1-Gossip] 2019-06-05 > 08:44:49,621 OutboundTcpConnection.java:486 - Seed gossip version is 8; will > not connect with that version > WARN [MessagingService-Outgoing-dc2_03/10.20.20.5-Gossip] 2019-06-05 > 08:44:49,621 OutboundTcpConnection.java:486 - Seed gossip version is 8; will > not connect with that version > WARN [GossipTasks:1] 2019-06-05 08:44:51,631 FailureDetector.java:278 - Not > marking nodes down due to local pause of 30943606906 > 5000000000 > > We've naturally stopped the upgrade but we still wish to upgrade from 2.1.21 > and hopefully find the root cause of this matter. > I'll be happy to provide additional details if needs be. > > -- This message was sent by Atlassian JIRA (v7.6.14#76016) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org