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

Anne Sullivan commented on CASSANDRA-6666:
------------------------------------------

Attached debug logs (cassandra_system.log.debug.gz)
- two node cluster, RF=2 (node A / node B)
- bring node B down for 30 minutes, back online @ 2014-03-19 10:15:00
- hinted handoff starts @  2014-03-19 10:18:00
- hinted handoff finishes successfully @ 2014-03-19 10:59:04:

INFO [HintedHandoff:1] 2014-03-19 10:59:04,434 HintedHandOffManager.java (line 
378) Finished hinted handoff of 1903040 rows to endpoint /###.###.###.###

- @ 2014-03-19 10:59:16,428 user defined compaction compacts down to one 
sstable (expected result = no sstables / empty set):

INFO [CompactionExecutor:3421] 2014-03-19 10:59:16,428 CompactionTask.java 
(line 275) Compacted 4 sstables to 
[/path/to/db/data/system/hints/system-hints-jb-49,].  172,043,512 bytes to 
6,365,114 (~3% of original) in 11,991ms = 0.506233MB/s.  4 total partitions 
merged to 1.  Partition merge counts were {4:1, }

- @ 2014-03-19 11:08:08 another hinted handoff starts (presumably because the 
hints table isn't empty, and it's checking every 10 minutes) and reports 0 live 
cells:

WARN [HintedHandoff:2] 2014-03-19 11:08:08,639 SliceQueryFilter.java (line 210) 
Read 0 live and 75750 tombstoned cells in system.hints (see 
tombstone_warn_threshold)
 INFO [HintedHandoff:2] 2014-03-19 11:08:08,639 HintedHandOffManager.java (line 
378) Finished hinted handoff of 0 rows to endpoint /###.###.###.###

- user defined compaction again leaves behind a single sstable:

 INFO [CompactionExecutor:3438] 2014-03-19 11:08:08,963 CompactionTask.java 
(line 275) Compacted 1 sstables to 
[/path/to/db/data/system/hints/system-hints-jb-50,].  6,365,114 bytes to 
6,365,114 (~100% of original) in 322ms = 18.851695MB/s.  1 total partitions 
merged to 1.  Partition merge counts were {1:1, }


Partial output from running "cqlsh> describe keyspace system", to confirm that 
gc_grace_seconds=0, so expecting tombstones to be removed on compaction:

CREATE TABLE hints (
  target_id uuid,
  hint_id timeuuid,
  message_version int,
  mutation blob,
  PRIMARY KEY (target_id, hint_id, message_version)
) WITH COMPACT STORAGE AND
  bloom_filter_fp_chance=0.010000 AND
  caching='KEYS_ONLY' AND
  comment='hints awaiting delivery' AND
  dclocal_read_repair_chance=0.000000 AND
  gc_grace_seconds=0 AND
  index_interval=128 AND
  read_repair_chance=0.000000 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='99.0PERCENTILE' AND
  memtable_flush_period_in_ms=3600000 AND
  compaction={'enabled': 'false', 'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'LZ4Compressor'};





> Avoid accumulating tombstones after partial hint replay
> -------------------------------------------------------
>
>                 Key: CASSANDRA-6666
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6666
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Jonathan Ellis
>            Assignee: Jonathan Ellis
>            Priority: Minor
>              Labels: hintedhandoff
>             Fix For: 1.2.16, 2.0.6
>
>         Attachments: 6666.txt, cassandra_system.log.debug.gz
>
>




--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to