[
https://issues.apache.org/jira/browse/CASSANDRA-4446?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13556790#comment-13556790
]
Robert Coli commented on CASSANDRA-4446:
----------------------------------------
While I'm sure that this does fix one real cause of drain not working in trunk
(yay!), one of the symptoms I've heard reported in the 1.0.x - 1.1.5 timeframe
is that "my counters over-counted on upgrade, despite drain". Most recent
report was 1.0.12->1.1.8 with drain being run as part of the upgrade process.
NEWS.txt says :
"If you using counters and upgrading from a version prior to 1.1.6, you should
drain existing Cassandra nodes prior to the upgrade to prevent overcount during
commitlog replay (see CASSANDRA-4782). For non-counter uses, drain is not
required but is a good practice to minimize restart time."
If drain in these versions can't be counted on (heh) to actually work for this
purpose (which reports suggest it cannot), then I propose changing this line to
read "drain existing nodes and remove their commitlog".
> nodetool drain sometimes doesn't mark commitlog fully flushed
> -------------------------------------------------------------
>
> Key: CASSANDRA-4446
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4446
> Project: Cassandra
> Issue Type: Bug
> Components: Core, Tools
> Environment: ubuntu 10.04 64bit
> Linux HOSTNAME 2.6.32-345-ec2 #48-Ubuntu SMP Wed May 2 19:29:55 UTC 2012
> x86_64 GNU/Linux
> sun JVM
> cassandra 1.0.10 installed from apache deb
> Reporter: Robert Coli
> Assignee: Jonathan Ellis
> Priority: Minor
> Fix For: 1.2.1
>
> Attachments: 4446.txt,
> cassandra.1.0.10.replaying.log.after.exception.during.drain.txt
>
>
> I recently wiped a customer's QA cluster. I drained each node and verified
> that they were drained. When I restarted the nodes, I saw the commitlog
> replay create a memtable and then flush it. I have attached a sanitized log
> snippet from a representative node at the time.
> It appears to show the following :
> 1) Drain begins
> 2) Drain triggers flush
> 3) Flush triggers compaction
> 4) StorageService logs DRAINED message
> 5) compaction thread excepts
> 6) on restart, same CF creates a memtable
> 7) and then flushes it [1]
> The columnfamily involved in the replay in 7) is the CF for which the
> compaction thread excepted in 5). This seems to suggest a timing issue
> whereby the exception in 5) prevents the flush in 3) from marking all the
> segments flushed, causing them to replay after restart.
> In case it might be relevant, I did an online change of compaction strategy
> from Leveled to SizeTiered during the uptime period preceding this drain.
> [1] Isn't commitlog replay not supposed to automatically trigger a flush in
> modern cassandra?
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira