Hi Jeff, I ran the scrub online and that didn't help. I went ahead and stopped the node, deleted all the corrupted data files <cf>-<num>-*.db files and planned on running a repair when it came back online.
Unrelated I believe, now another CF is corrupted! org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: /ephemeral/cassandra/data/OpsCenter/rollups300-45c85324387b35238d056678f8fa8b0f/OpsCenter-rollups300-ka-100672-Data.db Caused by: org.apache.cassandra.io.compress.CorruptBlockException: (/ephemeral/cassandra/data/OpsCenter/rollups300-45c85324387b35238d056678f8fa8b0f/OpsCenter-rollups300-ka-100672-Data.db): corruption detected, chunk at 101500 of length 26523398. Few days ago when troubleshooting this I did change the OpsCenter keyspace RF == 2 from 3 since I thought that would help reduce load. Did that cause this corruption? running *'nodetool scrub OpsCenter rollups300'* on that node now And now I also see this when running nodetool status: *"Note: Non-system keyspaces don't have the same replication settings, effective ownership information is meaningless"* What to do? I still can't stream to this new node cause of this corruption. Disk space is getting low on these nodes ... On Sat, Aug 12, 2017 at 9:51 PM Brian Spindler <brian.spind...@gmail.com> wrote: > nothing in logs on the node that it was streaming from. > > however, I think I found the issue on the other node in the C rack: > > ERROR [STREAM-IN-/10.40.17.114] 2017-08-12 16:48:53,354 > StreamSession.java:512 - [Stream #08957970-7f7e-11e7-b2a2-a31e21b877e5] > Streaming error occurred > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: > /ephemeral/cassandra/data/... > > I did a 'cat /var/log/cassandra/system.log|grep Corrupt' and it seems > it's a single Index.db file and nothing on the other node. > > I think nodetool scrub or offline sstablescrub might be in order but with > the current load I'm not sure I can take it offline for very long. > > Thanks again for the help. > > > On Sat, Aug 12, 2017 at 9:38 PM Jeffrey Jirsa <jji...@gmail.com> wrote: > >> Compaction is backed up – that may be normal write load (because of the >> rack imbalance), or it may be a secondary index build. Hard to say for >> sure. ‘nodetool compactionstats’ if you’re able to provide it. The jstack >> probably not necessary, streaming is being marked as failed and it’s >> turning itself off. Not sure why streaming is marked as failing, though, >> anything on the sending sides? >> >> >> >> >> >> From: Brian Spindler <brian.spind...@gmail.com> >> Reply-To: <user@cassandra.apache.org> >> Date: Saturday, August 12, 2017 at 6:34 PM >> To: <user@cassandra.apache.org> >> Subject: Re: Dropping down replication factor >> >> Thanks for replying Jeff. >> >> Responses below. >> >> On Sat, Aug 12, 2017 at 8:33 PM Jeff Jirsa <jji...@gmail.com> wrote: >> >>> Answers inline >>> >>> -- >>> Jeff Jirsa >>> >>> >>> > On Aug 12, 2017, at 2:58 PM, brian.spind...@gmail.com wrote: >>> > >>> > Hi folks, hopefully a quick one: >>> > >>> > We are running a 12 node cluster (2.1.15) in AWS with Ec2Snitch. It's >>> all in one region but spread across 3 availability zones. It was nicely >>> balanced with 4 nodes in each. >>> > >>> > But with a couple of failures and subsequent provisions to the wrong >>> az we now have a cluster with : >>> > >>> > 5 nodes in az A >>> > 5 nodes in az B >>> > 2 nodes in az C >>> > >>> > Not sure why, but when adding a third node in AZ C it fails to stream >>> after getting all the way to completion and no apparent error in logs. >>> I've looked at a couple of bugs referring to scrubbing and possible OOM >>> bugs due to metadata writing at end of streaming (sorry don't have ticket >>> handy). I'm worried I might not be able to do much with these since the >>> disk space usage is high and they are under a lot of load given the small >>> number of them for this rack. >>> >>> You'll definitely have higher load on az C instances with rf=3 in this >>> ratio >> >> >>> Streaming should still work - are you sure it's not busy doing >>> something? Like building secondary index or similar? jstack thread dump >>> would be useful, or at least nodetool tpstats >>> >>> Only other thing might be a backup. We do incrementals x1hr and >> snapshots x24h; they are shipped to s3 then links are cleaned up. The >> error I get on the node I'm trying to add to rack C is: >> >> ERROR [main] 2017-08-12 23:54:51,546 CassandraDaemon.java:583 - Exception >> encountered during startup >> java.lang.RuntimeException: Error during boostrap: Stream failed >> at >> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:87) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1166) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:944) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.StorageService.initServer(StorageService.java:740) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.StorageService.initServer(StorageService.java:617) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:391) >> [apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:566) >> [apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:655) >> [apache-cassandra-2.1.15.jar:2.1.15] >> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed >> at >> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) >> ~[guava-16.0.jar:na] >> at >> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) >> ~[guava-16.0.jar:na] >> at >> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) >> ~[guava-16.0.jar:na] >> at >> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) >> ~[guava-16.0.jar:na] >> at >> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) >> ~[guava-16.0.jar:na] >> at >> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:209) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:185) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:413) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:700) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:661) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:179) >> ~[apache-cassandra-2.1.15.jar:2.1.15] >> at >> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) >> ~[na:1.8.0_112] >> at java.util.concurrent.FutureTask.run(FutureTask.java:266) >> ~[na:1.8.0_112] >> at >> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) >> ~[na:1.8.0_112] >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) >> ~[na:1.8.0_112] >> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_112] >> WARN [StorageServiceShutdownHook] 2017-08-12 23:54:51,582 >> Gossiper.java:1462 - No local state or state is in silent shutdown, not >> announcing shutdown >> INFO [StorageServiceShutdownHook] 2017-08-12 23:54:51,582 >> MessagingService.java:734 - Waiting for messaging service to quiesce >> INFO [ACCEPT-/10.40.17.114] 2017-08-12 23:54:51,583 >> MessagingService.java:1020 - MessagingService has terminated the accept() >> thread >> >> And I got this on this same node when it was bootstrapping, I ran >> 'nodetool netstats' just before it shutdown: >> >> Receiving 377 files, 161928296443 bytes total. Already received >> 377 files, 161928296443 bytes total >> >> TPStats on host that was streaming the data to this node: >> >> Pool Name Active Pending Completed Blocked >> All time blocked >> MutationStage 1 1 4488289014 0 >> 0 >> ReadStage 0 0 24486526 0 >> 0 >> RequestResponseStage 0 0 3038847374 >> <(303)%20884-7374> 0 0 >> ReadRepairStage 0 0 1601576 0 >> 0 >> CounterMutationStage 0 0 68403 0 >> 0 >> MiscStage 0 0 0 0 >> 0 >> AntiEntropySessions 0 0 0 0 >> 0 >> HintedHandoff 0 0 18 0 >> 0 >> GossipStage 0 0 2786892 0 >> 0 >> CacheCleanupExecutor 0 0 0 0 >> 0 >> InternalResponseStage 0 0 61115 0 >> 0 >> CommitLogArchiver 0 0 0 0 >> 0 >> CompactionExecutor 4 83 304167 0 >> 0 >> ValidationExecutor 0 0 78249 0 >> 0 >> MigrationStage 0 0 94201 0 >> 0 >> AntiEntropyStage 0 0 160505 0 >> 0 >> PendingRangeCalculator 0 0 30 0 >> 0 >> Sampler 0 0 0 0 >> 0 >> MemtableFlushWriter 0 0 71270 0 >> 0 >> MemtablePostFlush 0 0 175209 0 >> 0 >> MemtableReclaimMemory 0 0 81222 0 >> 0 >> Native-Transport-Requests 2 0 1983565628 0 >> 9405444 >> >> Message type Dropped >> READ 218 >> RANGE_SLICE 15 >> _TRACE 0 >> MUTATION 2949001 >> COUNTER_MUTATION 0 >> BINARY 0 >> REQUEST_RESPONSE 0 >> PAGED_RANGE 0 >> READ_REPAIR 8571 >> >> I can get a jstack if needed. >> >>> >>> > >>> > Rather than troubleshoot this further, what I was thinking about doing >>> was: >>> > - drop the replication factor on our keyspace to two >>> >>> Repair before you do this, or you'll lose your consistency guarantees >>> >> >> Given the load on the 2 nodes in rack C I'm hoping a repair will succeed. >> >> >>> > - hopefully this would reduce load on these two remaining nodes >>> >>> It should, racks awareness guarantees on replica per rack if rf==num >>> racks, so right now those 2 c machines have 2.5x as much data as the >>> others. This will drop that requirement and drop the load significantly >>> >>> > - run repairs/cleanup across the cluster >>> > - then shoot these two nodes in the 'c' rack >>> >>> Why shoot the c instances? Why not drop RF and then add 2 more C >>> instances, then increase RF back to 3, run repair, then Decom the extra >>> instances in a and b? >>> >>> >>> Fair point. I was considering staying at RF two but I think with your >> points below, I should reconsider. >> >> >>> > - run repairs/cleanup across the cluster >>> > >>> > Would this work with minimal/no disruption? >>> >>> The big risk of running rf=2 is that quorum==all - any gc pause or node >>> restarting will make you lose HA or strong consistency guarantees. >>> >>> > Should I update their "rack" before hand or after ? >>> >>> You can't change a node's rack once it's in the cluster, it SHOULD >>> refuse to start if you do that >>> >>> Got it. >> >> >>> > What else am I not thinking about? >>> > >>> > My main goal atm is to get back to where the cluster is in a clean >>> consistent state that allows nodes to properly bootstrap. >>> > >>> > Thanks for your help in advance. >>> > --------------------------------------------------------------------- >>> > To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org >>> > For additional commands, e-mail: user-h...@cassandra.apache.org >>> > >>> >>> --------------------------------------------------------------------- >>> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org >>> For additional commands, e-mail: user-h...@cassandra.apache.org >>> >>>