rocksdb/utilities/cassandra/cassandra_compaction_filter.h
Pengchao Wang e4234fbdcf collecting kValue type tombstone
Summary:
In our testing cluster, we found large amount tombstone has been promoted to kValue type from kMerge after reaching the top level of compaction. Since we used to only collecting tombstone in merge operator, those tombstones can never be collected.

This PR addresses the issue by adding a GC step in compaction filter, which is only for kValue type records. Since those record already reached the top of compaction (no earlier data exists) we can safely remove them in compaction filter without worrying old data appears.

This PR also removes an old optimization in cassandra merge operator for single merge operands.  We need to do GC even on a single operand, so the optimation does not make sense anymore.
Closes https://github.com/facebook/rocksdb/pull/2855

Reviewed By: sagar0

Differential Revision: D5806445

Pulled By: wpc

fbshipit-source-id: 6eb25629d4ce917eb5e8b489f64a6aa78c7d270b
2017-09-18 16:27:12 -07:00

43 lines
1.7 KiB
C++

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#pragma once
#include <string>
#include "rocksdb/compaction_filter.h"
#include "rocksdb/slice.h"
namespace rocksdb {
namespace cassandra {
/**
* Compaction filter for removing expired Cassandra data with ttl.
* If option `purge_ttl_on_expiration` is set to true, expired data
* will be directly purged. Otherwise expired data will be converted
* tombstones first, then be eventally removed after gc grace period.
* `purge_ttl_on_expiration` should only be on in the case all the
* writes have same ttl setting, otherwise it could bring old data back.
*
* Compaction filter is also in charge of removing tombstone that has been
* promoted to kValue type after serials of merging in compaction.
*/
class CassandraCompactionFilter : public CompactionFilter {
public:
explicit CassandraCompactionFilter(bool purge_ttl_on_expiration,
int32_t gc_grace_period_in_seconds)
: purge_ttl_on_expiration_(purge_ttl_on_expiration),
gc_grace_period_in_seconds_(gc_grace_period_in_seconds) {}
const char* Name() const override;
virtual Decision FilterV2(int level, const Slice& key, ValueType value_type,
const Slice& existing_value, std::string* new_value,
std::string* skip_until) const override;
private:
bool purge_ttl_on_expiration_;
int32_t gc_grace_period_in_seconds_;
};
} // namespace cassandra
} // namespace rocksdb