Unverified Commit e9b74b88 authored by yiwu-arbug's avatar yiwu-arbug Committed by GitHub

Add SST partitioner interface (#512)

Add interface for `SstPartitioner`. The interface is to support implementing "compaction guard" to split SST by custom boundaries.
Signed-off-by: 's avatarYi Wu <yiwu@pingcap.com>
parent ee6c08ad
......@@ -31,5 +31,6 @@ librocksdb_sys = { path = "librocksdb_sys" }
[dev-dependencies]
crc = "1.8"
lazy_static = "1.4.0"
rand = "0.7"
tempfile = "3.1"
......@@ -2,7 +2,7 @@ all: format build test
format:
@cargo fmt --all
@librocksdb_sys/crocksdb/format-diff.sh > /dev/null || true
@scripts/format-diff.sh
build:
@cargo build
......
......@@ -36,6 +36,7 @@
#include "rocksdb/slice_transform.h"
#include "rocksdb/sst_dump_tool.h"
#include "rocksdb/sst_file_reader.h"
#include "rocksdb/sst_partitioner.h"
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
#include "rocksdb/table.h"
......@@ -115,6 +116,8 @@ using rocksdb::NewEncryptedEnv;
using rocksdb::NewGenericRateLimiter;
using rocksdb::NewLRUCache;
using rocksdb::Options;
using rocksdb::PartitionerRequest;
using rocksdb::PartitionerResult;
using rocksdb::PinnableSlice;
using rocksdb::RandomAccessFile;
using rocksdb::Range;
......@@ -130,6 +133,8 @@ using rocksdb::SliceTransform;
using rocksdb::Snapshot;
using rocksdb::SstFileReader;
using rocksdb::SstFileWriter;
using rocksdb::SstPartitioner;
using rocksdb::SstPartitionerFactory;
using rocksdb::Status;
using rocksdb::TableProperties;
using rocksdb::TablePropertiesCollection;
......@@ -637,6 +642,24 @@ struct crocksdb_encryption_key_manager_t {
};
#endif
struct crocksdb_sst_partitioner_t {
std::unique_ptr<SstPartitioner> rep;
};
struct crocksdb_sst_partitioner_request_t {
PartitionerRequest* rep;
Slice prev_user_key;
Slice current_user_key;
};
struct crocksdb_sst_partitioner_context_t {
SstPartitioner::Context* rep;
};
struct crocksdb_sst_partitioner_factory_t {
std::shared_ptr<SstPartitionerFactory> rep;
};
static bool SaveError(char** errptr, const Status& s) {
assert(errptr != nullptr);
if (s.ok()) {
......@@ -2490,6 +2513,19 @@ void crocksdb_options_set_max_bytes_for_level_multiplier_additional(
}
}
crocksdb_sst_partitioner_factory_t*
crocksdb_options_get_sst_partitioner_factory(crocksdb_options_t* opt) {
crocksdb_sst_partitioner_factory_t* factory =
new crocksdb_sst_partitioner_factory_t;
factory->rep = opt->rep.sst_partitioner_factory;
return factory;
}
void crocksdb_options_set_sst_partitioner_factory(
crocksdb_options_t* opt, crocksdb_sst_partitioner_factory_t* factory) {
opt->rep.sst_partitioner_factory = factory->rep;
}
void crocksdb_options_enable_statistics(crocksdb_options_t* opt,
unsigned char v) {
if (v) {
......@@ -5631,6 +5667,255 @@ uint64_t crocksdb_iostats_context_logger_nanos(
return ctx->rep.logger_nanos;
}
crocksdb_sst_partitioner_request_t* crocksdb_sst_partitioner_request_create() {
auto* req = new crocksdb_sst_partitioner_request_t;
req->rep =
new PartitionerRequest(req->prev_user_key, req->current_user_key, 0);
return req;
}
void crocksdb_sst_partitioner_request_destroy(
crocksdb_sst_partitioner_request_t* req) {
delete req->rep;
delete req;
}
const char* crocksdb_sst_partitioner_request_prev_user_key(
crocksdb_sst_partitioner_request_t* req, size_t* len) {
const Slice* prev_key = req->rep->prev_user_key;
*len = prev_key->size();
return prev_key->data();
}
const char* crocksdb_sst_partitioner_request_current_user_key(
crocksdb_sst_partitioner_request_t* req, size_t* len) {
const Slice* current_key = req->rep->current_user_key;
*len = current_key->size();
return current_key->data();
}
uint64_t crocksdb_sst_partitioner_request_current_output_file_size(
crocksdb_sst_partitioner_request_t* req) {
return req->rep->current_output_file_size;
}
void crocksdb_sst_partitioner_request_set_prev_user_key(
crocksdb_sst_partitioner_request_t* req, const char* key, size_t len) {
req->prev_user_key = Slice(key, len);
req->rep->prev_user_key = &req->prev_user_key;
}
void crocksdb_sst_partitioner_request_set_current_user_key(
crocksdb_sst_partitioner_request_t* req, const char* key, size_t len) {
req->current_user_key = Slice(key, len);
req->rep->current_user_key = &req->current_user_key;
}
void crocksdb_sst_partitioner_request_set_current_output_file_size(
crocksdb_sst_partitioner_request_t* req,
uint64_t current_output_file_size) {
req->rep->current_output_file_size = current_output_file_size;
}
struct crocksdb_sst_partitioner_impl_t : public SstPartitioner {
void* underlying;
void (*destructor)(void*);
crocksdb_sst_partitioner_should_partition_cb should_partition_cb;
crocksdb_sst_partitioner_can_do_trivial_move_cb can_do_trivial_move_cb;
virtual ~crocksdb_sst_partitioner_impl_t() { destructor(underlying); }
const char* Name() const override { return "crocksdb_sst_partitioner_impl"; }
PartitionerResult ShouldPartition(
const PartitionerRequest& request) override {
crocksdb_sst_partitioner_request_t req;
req.rep = const_cast<PartitionerRequest*>(&request);
return static_cast<PartitionerResult>(
should_partition_cb(underlying, &req));
}
bool CanDoTrivialMove(const Slice& smallest_user_key,
const Slice& largest_user_key) override {
return can_do_trivial_move_cb(
underlying, smallest_user_key.data(), smallest_user_key.size(),
largest_user_key.data(), largest_user_key.size());
}
};
crocksdb_sst_partitioner_t* crocksdb_sst_partitioner_create(
void* underlying, void (*destructor)(void*),
crocksdb_sst_partitioner_should_partition_cb should_partition_cb,
crocksdb_sst_partitioner_can_do_trivial_move_cb can_do_trivial_move_cb) {
crocksdb_sst_partitioner_impl_t* sst_partitioner_impl =
new crocksdb_sst_partitioner_impl_t;
sst_partitioner_impl->underlying = underlying;
sst_partitioner_impl->destructor = destructor;
sst_partitioner_impl->should_partition_cb = should_partition_cb;
sst_partitioner_impl->can_do_trivial_move_cb = can_do_trivial_move_cb;
crocksdb_sst_partitioner_t* sst_partitioner = new crocksdb_sst_partitioner_t;
sst_partitioner->rep.reset(sst_partitioner_impl);
return sst_partitioner;
}
void crocksdb_sst_partitioner_destroy(crocksdb_sst_partitioner_t* partitioner) {
delete partitioner;
}
crocksdb_sst_partitioner_result_t crocksdb_sst_partitioner_should_partition(
crocksdb_sst_partitioner_t* partitioner,
crocksdb_sst_partitioner_request_t* req) {
return static_cast<crocksdb_sst_partitioner_result_t>(
partitioner->rep->ShouldPartition(*req->rep));
}
unsigned char crocksdb_sst_partitioner_can_do_trivial_move(
crocksdb_sst_partitioner_t* partitioner, const char* smallest_user_key,
size_t smallest_user_key_len, const char* largest_user_key,
size_t largest_user_key_len) {
Slice smallest_key(smallest_user_key, smallest_user_key_len);
Slice largest_key(largest_user_key, largest_user_key_len);
return partitioner->rep->CanDoTrivialMove(smallest_key, largest_key);
}
crocksdb_sst_partitioner_context_t* crocksdb_sst_partitioner_context_create() {
auto* rep = new SstPartitioner::Context;
auto* context = new crocksdb_sst_partitioner_context_t;
context->rep = rep;
return context;
}
void crocksdb_sst_partitioner_context_destroy(
crocksdb_sst_partitioner_context_t* context) {
delete context->rep;
delete context;
}
unsigned char crocksdb_sst_partitioner_context_is_full_compaction(
crocksdb_sst_partitioner_context_t* context) {
return context->rep->is_full_compaction;
}
unsigned char crocksdb_sst_partitioner_context_is_manual_compaction(
crocksdb_sst_partitioner_context_t* context) {
return context->rep->is_manual_compaction;
}
int crocksdb_sst_partitioner_context_output_level(
crocksdb_sst_partitioner_context_t* context) {
return context->rep->output_level;
}
const char* crocksdb_sst_partitioner_context_smallest_key(
crocksdb_sst_partitioner_context_t* context, size_t* key_len) {
auto& smallest_key = context->rep->smallest_user_key;
*key_len = smallest_key.size();
return smallest_key.data();
}
const char* crocksdb_sst_partitioner_context_largest_key(
crocksdb_sst_partitioner_context_t* context, size_t* key_len) {
auto& largest_key = context->rep->largest_user_key;
*key_len = largest_key.size();
return largest_key.data();
}
void crocksdb_sst_partitioner_context_set_is_full_compaction(
crocksdb_sst_partitioner_context_t* context,
unsigned char is_full_compaction) {
context->rep->is_full_compaction = is_full_compaction;
}
void crocksdb_sst_partitioner_context_set_is_manual_compaction(
crocksdb_sst_partitioner_context_t* context,
unsigned char is_manual_compaction) {
context->rep->is_manual_compaction = is_manual_compaction;
}
void crocksdb_sst_partitioner_context_set_output_level(
crocksdb_sst_partitioner_context_t* context, int output_level) {
context->rep->output_level = output_level;
}
void crocksdb_sst_partitioner_context_set_smallest_key(
crocksdb_sst_partitioner_context_t* context, const char* smallest_key,
size_t key_len) {
context->rep->smallest_user_key = Slice(smallest_key, key_len);
}
void crocksdb_sst_partitioner_context_set_largest_key(
crocksdb_sst_partitioner_context_t* context, const char* largest_key,
size_t key_len) {
context->rep->largest_user_key = Slice(largest_key, key_len);
}
struct crocksdb_sst_partitioner_factory_impl_t : public SstPartitionerFactory {
void* underlying;
void (*destructor)(void*);
crocksdb_sst_partitioner_factory_name_cb name_cb;
crocksdb_sst_partitioner_factory_create_partitioner_cb create_partitioner_cb;
virtual ~crocksdb_sst_partitioner_factory_impl_t() { destructor(underlying); }
const char* Name() const override { return name_cb(underlying); }
std::unique_ptr<SstPartitioner> CreatePartitioner(
const SstPartitioner::Context& partitioner_context) const override {
crocksdb_sst_partitioner_context_t context;
context.rep = const_cast<SstPartitioner::Context*>(&partitioner_context);
crocksdb_sst_partitioner_t* partitioner =
create_partitioner_cb(underlying, &context);
if (partitioner == nullptr) {
return nullptr;
}
std::unique_ptr<SstPartitioner> rep = std::move(partitioner->rep);
crocksdb_sst_partitioner_destroy(partitioner);
return rep;
}
};
crocksdb_sst_partitioner_factory_t* crocksdb_sst_partitioner_factory_create(
void* underlying, void (*destructor)(void*),
crocksdb_sst_partitioner_factory_name_cb name_cb,
crocksdb_sst_partitioner_factory_create_partitioner_cb
create_partitioner_cb) {
crocksdb_sst_partitioner_factory_impl_t* factory_impl =
new crocksdb_sst_partitioner_factory_impl_t;
factory_impl->underlying = underlying;
factory_impl->destructor = destructor;
factory_impl->name_cb = name_cb;
factory_impl->create_partitioner_cb = create_partitioner_cb;
crocksdb_sst_partitioner_factory_t* factory =
new crocksdb_sst_partitioner_factory_t;
factory->rep.reset(factory_impl);
return factory;
}
void crocksdb_sst_partitioner_factory_destroy(
crocksdb_sst_partitioner_factory_t* factory) {
delete factory;
}
const char* crocksdb_sst_partitioner_factory_name(
crocksdb_sst_partitioner_factory_t* factory) {
return factory->rep->Name();
}
crocksdb_sst_partitioner_t* crocksdb_sst_partitioner_factory_create_partitioner(
crocksdb_sst_partitioner_factory_t* factory,
crocksdb_sst_partitioner_context_t* context) {
std::unique_ptr<SstPartitioner> rep =
factory->rep->CreatePartitioner(*context->rep);
if (rep == nullptr) {
return nullptr;
}
crocksdb_sst_partitioner_t* partitioner = new crocksdb_sst_partitioner_t;
partitioner->rep = std::move(rep);
return partitioner;
}
/* Tools */
void crocksdb_run_ldb_tool(int argc, char** argv,
const crocksdb_options_t* opts) {
LDBTool().Run(argc, argv, opts->rep);
......
......@@ -154,6 +154,19 @@ typedef struct crocksdb_writestallcondition_t crocksdb_writestallcondition_t;
typedef struct crocksdb_map_property_t crocksdb_map_property_t;
typedef struct crocksdb_writebatch_iterator_t crocksdb_writebatch_iterator_t;
typedef enum crocksdb_sst_partitioner_result_t {
kNotRequired = 0,
kRequired = 1,
} crocksdb_sst_partitioner_result_t;
typedef struct crocksdb_sst_partitioner_t crocksdb_sst_partitioner_t;
typedef struct crocksdb_sst_partitioner_request_t
crocksdb_sst_partitioner_request_t;
typedef struct crocksdb_sst_partitioner_context_t
crocksdb_sst_partitioner_context_t;
typedef struct crocksdb_sst_partitioner_factory_t
crocksdb_sst_partitioner_factory_t;
typedef enum crocksdb_table_property_t {
kDataSize = 1,
kIndexSize = 2,
......@@ -1005,6 +1018,10 @@ crocksdb_options_get_max_bytes_for_level_multiplier(crocksdb_options_t*);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_options_set_max_bytes_for_level_multiplier_additional(
crocksdb_options_t*, int* level_values, size_t num_levels);
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_factory_t*
crocksdb_options_get_sst_partitioner_factory(crocksdb_options_t*);
extern C_ROCKSDB_LIBRARY_API void crocksdb_options_set_sst_partitioner_factory(
crocksdb_options_t*, crocksdb_sst_partitioner_factory_t*);
extern C_ROCKSDB_LIBRARY_API void crocksdb_options_enable_statistics(
crocksdb_options_t*, unsigned char);
extern C_ROCKSDB_LIBRARY_API void crocksdb_options_reset_statistics(
......@@ -2225,6 +2242,115 @@ crocksdb_iostats_context_prepare_write_nanos(crocksdb_iostats_context_t*);
extern C_ROCKSDB_LIBRARY_API uint64_t
crocksdb_iostats_context_logger_nanos(crocksdb_iostats_context_t*);
/* SstPartitioner */
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_request_t*
crocksdb_sst_partitioner_request_create();
extern C_ROCKSDB_LIBRARY_API void crocksdb_sst_partitioner_request_destroy(
crocksdb_sst_partitioner_request_t* req);
extern C_ROCKSDB_LIBRARY_API const char*
crocksdb_sst_partitioner_request_prev_user_key(
crocksdb_sst_partitioner_request_t* req, size_t* len);
extern C_ROCKSDB_LIBRARY_API const char*
crocksdb_sst_partitioner_request_current_user_key(
crocksdb_sst_partitioner_request_t* req, size_t* len);
extern C_ROCKSDB_LIBRARY_API uint64_t
crocksdb_sst_partitioner_request_current_output_file_size(
crocksdb_sst_partitioner_request_t* req);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_req_set_prev_user_key(
crocksdb_sst_partitioner_request_t* req, const char* key, size_t len);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_req_set_current_user_key(
crocksdb_sst_partitioner_request_t* req, const char* key, size_t len);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_request_set_current_output_file_size(
crocksdb_sst_partitioner_request_t* req, uint64_t current_output_file_size);
typedef crocksdb_sst_partitioner_result_t (
*crocksdb_sst_partitioner_should_partition_cb)(
void* underlying, crocksdb_sst_partitioner_request_t* req);
typedef unsigned char (*crocksdb_sst_partitioner_can_do_trivial_move_cb)(
void* underlying, const char* smallest_user_key,
size_t smallest_user_key_len, const char* largest_user_key,
size_t largest_user_key_len);
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_t*
crocksdb_sst_partitioner_create(
void* underlying, void (*destructor)(void*),
crocksdb_sst_partitioner_should_partition_cb should_partition_cb,
crocksdb_sst_partitioner_can_do_trivial_move_cb can_do_trivial_move_cb);
extern C_ROCKSDB_LIBRARY_API void crocksdb_sst_partitioner_destroy(
crocksdb_sst_partitioner_t* partitioner);
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_result_t
crocksdb_sst_partitioner_should_partition(
crocksdb_sst_partitioner_t* partitioner,
crocksdb_sst_partitioner_request_t* req);
extern C_ROCKSDB_LIBRARY_API unsigned char
crocksdb_sst_partitioner_can_do_trivial_move(
crocksdb_sst_partitioner_t* partitioner, const char* smallest_user_key,
size_t smallest_user_key_len, const char* largest_user_key,
size_t largest_user_key_len);
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_context_t*
crocksdb_sst_partitioner_context_create();
extern C_ROCKSDB_LIBRARY_API void crocksdb_sst_partitioner_context_destroy(
crocksdb_sst_partitioner_context_t* context);
extern C_ROCKSDB_LIBRARY_API unsigned char
crocksdb_sst_partitioner_context_is_full_compaction(
crocksdb_sst_partitioner_context_t* context);
extern C_ROCKSDB_LIBRARY_API unsigned char
crocksdb_sst_partitioner_context_is_manual_compaction(
crocksdb_sst_partitioner_context_t* context);
extern C_ROCKSDB_LIBRARY_API int crocksdb_sst_partitioner_context_output_level(
crocksdb_sst_partitioner_context_t* context);
extern C_ROCKSDB_LIBRARY_API const char*
crocksdb_sst_partitioner_context_smallest_key(
crocksdb_sst_partitioner_context_t* context, size_t* key_len);
extern C_ROCKSDB_LIBRARY_API const char*
crocksdb_sst_partitioner_context_largest_key(
crocksdb_sst_partitioner_context_t* context, size_t* key_len);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_context_set_is_full_compaction(
crocksdb_sst_partitioner_context_t* context,
unsigned char is_full_compaction);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_context_set_is_manual_compaction(
crocksdb_sst_partitioner_context_t* context,
unsigned char is_manual_compaction);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_context_set_output_level(
crocksdb_sst_partitioner_context_t* context, int output_level);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_context_set_smallest_key(
crocksdb_sst_partitioner_context_t* context, const char* smallest_key,
size_t key_len);
extern C_ROCKSDB_LIBRARY_API void
crocksdb_sst_partitioner_context_set_largest_key(
crocksdb_sst_partitioner_context_t* context, const char* largest_key,
size_t key_len);
typedef const char* (*crocksdb_sst_partitioner_factory_name_cb)(
void* underlying);
typedef crocksdb_sst_partitioner_t* (
*crocksdb_sst_partitioner_factory_create_partitioner_cb)(
void* underlying, crocksdb_sst_partitioner_context_t* context);
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_factory_t*
crocksdb_sst_partitioner_factory_create(
void* underlying, void (*destructor)(void*),
crocksdb_sst_partitioner_factory_name_cb name_cb,
crocksdb_sst_partitioner_factory_create_partitioner_cb
create_partitioner_cb);
extern C_ROCKSDB_LIBRARY_API void crocksdb_sst_partitioner_factory_destroy(
crocksdb_sst_partitioner_factory_t* factory);
extern C_ROCKSDB_LIBRARY_API const char* crocksdb_sst_partitioner_factory_name(
crocksdb_sst_partitioner_factory_t* factory);
extern C_ROCKSDB_LIBRARY_API crocksdb_sst_partitioner_t*
crocksdb_sst_partitioner_factory_create_partitioner(
crocksdb_sst_partitioner_factory_t* factory,
crocksdb_sst_partitioner_context_t* context);
extern C_ROCKSDB_LIBRARY_API void crocksdb_run_ldb_tool(
int argc, char** argv, const crocksdb_options_t* opts);
extern C_ROCKSDB_LIBRARY_API void crocksdb_run_sst_dump_tool(
......
......@@ -164,6 +164,14 @@ pub struct DBFileEncryptionInfo(c_void);
#[repr(C)]
pub struct DBEncryptionKeyManagerInstance(c_void);
#[repr(C)]
pub struct DBSstPartitioner(c_void);
#[repr(C)]
pub struct DBSstPartitionerRequest(c_void);
#[repr(C)]
pub struct DBSstPartitionerContext(c_void);
#[repr(C)]
pub struct DBSstPartitionerFactory(c_void);
#[repr(C)]
pub struct DBWriteBatchIterator(c_void);
#[derive(Copy, Clone, Debug, Eq, PartialEq)]
......@@ -431,6 +439,13 @@ impl fmt::Display for DBEncryptionMethod {
}
}
#[derive(Copy, Clone, Debug, Eq, PartialEq)]
#[repr(C)]
pub enum DBSstPartitionerResult {
NotRequired = 0,
Required = 1,
}
/// # Safety
///
/// ptr must point to a valid CStr value
......@@ -825,6 +840,13 @@ extern "C" {
pub fn crocksdb_options_get_path_target_size(options: *mut Options, idx: size_t) -> u64;
pub fn crocksdb_options_set_vector_memtable_factory(options: *mut Options, reserved_bytes: u64);
pub fn crocksdb_options_set_atomic_flush(option: *mut Options, enable: bool);
pub fn crocksdb_options_get_sst_partitioner_factory(
option: *mut Options,
) -> *mut DBSstPartitionerFactory;
pub fn crocksdb_options_set_sst_partitioner_factory(
option: *mut Options,
factory: *mut DBSstPartitionerFactory,
);
pub fn crocksdb_filterpolicy_create_bloom_full(bits_per_key: c_int) -> *mut DBFilterPolicy;
pub fn crocksdb_filterpolicy_create_bloom(bits_per_key: c_int) -> *mut DBFilterPolicy;
pub fn crocksdb_open(
......@@ -2265,6 +2287,122 @@ extern "C" {
pub fn crocksdb_iostats_context_prepare_write_nanos(ctx: *mut DBIOStatsContext) -> u64;
pub fn crocksdb_iostats_context_logger_nanos(ctx: *mut DBIOStatsContext) -> u64;
pub fn crocksdb_sst_partitioner_request_create() -> *mut DBSstPartitionerRequest;
pub fn crocksdb_sst_partitioner_request_destroy(state: *mut DBSstPartitionerRequest);
pub fn crocksdb_sst_partitioner_request_prev_user_key(
state: *mut DBSstPartitionerRequest,
len: *mut size_t,
) -> *const c_char;
pub fn crocksdb_sst_partitioner_request_current_user_key(
state: *mut DBSstPartitionerRequest,
len: *mut size_t,
) -> *const c_char;
pub fn crocksdb_sst_partitioner_request_current_output_file_size(
state: *mut DBSstPartitionerRequest,
) -> u64;
pub fn crocksdb_sst_partitioner_request_set_prev_user_key(
state: *mut DBSstPartitionerRequest,
key: *const c_char,
len: size_t,
);
pub fn crocksdb_sst_partitioner_request_set_current_user_key(
state: *mut DBSstPartitionerRequest,
key: *const c_char,
len: size_t,
);
pub fn crocksdb_sst_partitioner_request_set_current_output_file_size(
state: *mut DBSstPartitionerRequest,
current_output_file_size: u64,
);
pub fn crocksdb_sst_partitioner_create(
underlying: *mut c_void,
destructor: extern "C" fn(*mut c_void),
should_partition_cb: extern "C" fn(
*mut c_void,
*mut DBSstPartitionerRequest,
) -> DBSstPartitionerResult,
can_do_trivial_move_cb: extern "C" fn(
*mut c_void,
*const c_char,
size_t,
*const c_char,
size_t,
) -> c_uchar,
) -> *mut DBSstPartitioner;
pub fn crocksdb_sst_partitioner_destroy(partitioner: *mut DBSstPartitioner);
pub fn crocksdb_sst_partitioner_should_partition(
partitioner: *mut DBSstPartitioner,
req: *mut DBSstPartitionerRequest,
) -> DBSstPartitionerResult;
pub fn crocksdb_sst_partitioner_can_do_trivial_move(
partitioner: *mut DBSstPartitioner,
smallest_key: *const c_char,
smallest_key_len: size_t,
largest_key: *const c_char,
largest_key_len: size_t,
) -> bool;
pub fn crocksdb_sst_partitioner_context_create() -> *mut DBSstPartitionerContext;
pub fn crocksdb_sst_partitioner_context_destroy(context: *mut DBSstPartitionerContext);
pub fn crocksdb_sst_partitioner_context_is_full_compaction(
context: *mut DBSstPartitionerContext,
) -> c_uchar;
pub fn crocksdb_sst_partitioner_context_is_manual_compaction(
context: *mut DBSstPartitionerContext,
) -> c_uchar;
pub fn crocksdb_sst_partitioner_context_output_level(
context: *mut DBSstPartitionerContext,
) -> c_int;
pub fn crocksdb_sst_partitioner_context_smallest_key(
context: *mut DBSstPartitionerContext,
key_len: *mut size_t,
) -> *const c_char;
pub fn crocksdb_sst_partitioner_context_largest_key(
context: *mut DBSstPartitionerContext,
key_len: *mut size_t,
) -> *const c_char;
pub fn crocksdb_sst_partitioner_context_set_is_full_compaction(
context: *mut DBSstPartitionerContext,
is_full_compaction: c_uchar,
);
pub fn crocksdb_sst_partitioner_context_set_is_manual_compaction(
context: *mut DBSstPartitionerContext,
is_manual_compaction: c_uchar,
);
pub fn crocksdb_sst_partitioner_context_set_output_level(
context: *mut DBSstPartitionerContext,
output_level: c_int,
);
pub fn crocksdb_sst_partitioner_context_set_smallest_key(
context: *mut DBSstPartitionerContext,
smallest_key: *const c_char,
key_len: size_t,
);
pub fn crocksdb_sst_partitioner_context_set_largest_key(
context: *mut DBSstPartitionerContext,
largest_key: *const c_char,
key_len: size_t,
);
pub fn crocksdb_sst_partitioner_factory_create(
underlying: *mut c_void,
destructor: extern "C" fn(*mut c_void),
name_cb: extern "C" fn(*mut c_void) -> *const c_char,
create_partitioner_cb: extern "C" fn(
*mut c_void,
*mut DBSstPartitionerContext,
) -> *mut DBSstPartitioner,
) -> *mut DBSstPartitionerFactory;
pub fn crocksdb_sst_partitioner_factory_destroy(factory: *mut DBSstPartitionerFactory);
pub fn crocksdb_sst_partitioner_factory_name(
factory: *mut DBSstPartitionerFactory,
) -> *const c_char;
pub fn crocksdb_sst_partitioner_factory_create_partitioner(
factory: *mut DBSstPartitionerFactory,
context: *mut DBSstPartitionerContext,
) -> *mut DBSstPartitioner;
pub fn crocksdb_run_ldb_tool(argc: c_int, argv: *const *const c_char, opts: *const Options);
pub fn crocksdb_run_sst_dump_tool(
argc: c_int,
......
nightly-2019-12-20
nightly-2020-07-01
......@@ -23,6 +23,9 @@ extern crate libc;
pub extern crate librocksdb_sys;
#[cfg(test)]
extern crate tempfile;
#[cfg(test)]
#[macro_use]
extern crate lazy_static;
#[cfg(feature = "cloud")]
pub use cloud::CloudEnvOptions;
......@@ -39,7 +42,8 @@ pub use event_listener::{
pub use librocksdb_sys::{
self as crocksdb_ffi, new_bloom_filter, CompactionPriority, CompactionReason,
DBBackgroundErrorReason, DBBottommostLevelCompaction, DBCompactionStyle, DBCompressionType,
DBEntryType, DBInfoLogLevel, DBRateLimiterMode, DBRecoveryMode, DBStatisticsHistogramType,
DBEntryType, DBInfoLogLevel, DBRateLimiterMode, DBRecoveryMode,
DBSstPartitionerResult as SstPartitionerResult, DBStatisticsHistogramType,
DBStatisticsTickerType, DBStatusPtr, DBTitanDBBlobRunMode, IndexType, WriteStallCondition,
};
pub use logger::Logger;
......@@ -58,6 +62,9 @@ pub use rocksdb_options::{
WriteOptions,
};
pub use slice_transform::SliceTransform;
pub use sst_partitioner::{
SstPartitioner, SstPartitionerContext, SstPartitionerFactory, SstPartitionerRequest,
};
pub use table_filter::TableFilter;
pub use table_properties::{
TableProperties, TablePropertiesCollection, TablePropertiesCollectionView,
......@@ -85,6 +92,7 @@ mod perf_context;
pub mod rocksdb;
pub mod rocksdb_options;
mod slice_transform;
pub mod sst_partitioner;
mod table_filter;
mod table_properties;
mod table_properties_collector;
......
......@@ -34,6 +34,7 @@ use merge_operator::{self, full_merge_callback, partial_merge_callback, MergeOpe
use rocksdb::Env;
use rocksdb::{Cache, MemoryAllocator};
use slice_transform::{new_slice_transform, SliceTransform};
use sst_partitioner::{new_sst_partitioner_factory, SstPartitionerFactory};
use std::ffi::{CStr, CString};
use std::path::Path;
use std::ptr;
......@@ -1740,6 +1741,13 @@ impl ColumnFamilyOptions {
Some(CStr::from_ptr(memtable_name).to_str().unwrap())
}
}
pub fn set_sst_partitioner_factory<F: SstPartitionerFactory>(&mut self, factory: F) {
let f = new_sst_partitioner_factory(factory);
unsafe {
crocksdb_ffi::crocksdb_options_set_sst_partitioner_factory(self.inner, f);
}
}
}
// ColumnFamilyDescriptor is a pair of column family's name and options.
......
// Copyright 2020 TiKV Project Authors. Licensed under Apache-2.0.
use super::SstPartitionerResult;
use crocksdb_ffi::{
self, DBSstPartitioner, DBSstPartitionerContext, DBSstPartitionerFactory,
DBSstPartitionerRequest,
};
use libc::{c_char, c_uchar, c_void, size_t};
use std::{ffi::CString, ptr, slice};
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct SstPartitionerRequest<'a> {
pub prev_user_key: &'a [u8],
pub current_user_key: &'a [u8],
pub current_output_file_size: u64,
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct SstPartitionerContext<'a> {
pub is_full_compaction: bool,
pub is_manual_compaction: bool,
pub output_level: i32,
pub smallest_key: &'a [u8],
pub largest_key: &'a [u8],
}
pub trait SstPartitioner {
fn should_partition(&self, req: &SstPartitionerRequest) -> SstPartitionerResult;
fn can_do_trivial_move(&self, smallest_user_key: &[u8], largest_user_key: &[u8]) -> bool;
}
extern "C" fn sst_partitioner_destructor<P: SstPartitioner>(ctx: *mut c_void) {
unsafe {
// Recover from raw pointer and implicitly drop.
Box::from_raw(ctx as *mut P);
}
}
extern "C" fn sst_partitioner_should_partition<P: SstPartitioner>(
ctx: *mut c_void,
request: *mut DBSstPartitionerRequest,
) -> SstPartitionerResult {
let partitioner = unsafe { &*(ctx as *mut P) };
let req = unsafe {
let mut prev_key_len: usize = 0;
let prev_key = crocksdb_ffi::crocksdb_sst_partitioner_request_prev_user_key(
request,
&mut prev_key_len,
) as *const u8;
let mut current_key_len: usize = 0;
let current_key = crocksdb_ffi::crocksdb_sst_partitioner_request_current_user_key(
request,
&mut current_key_len,
) as *const u8;
SstPartitionerRequest {
prev_user_key: slice::from_raw_parts(prev_key, prev_key_len),
current_user_key: slice::from_raw_parts(current_key, current_key_len),
current_output_file_size:
crocksdb_ffi::crocksdb_sst_partitioner_request_current_output_file_size(request),
}
};
partitioner.should_partition(&req) as _
}
extern "C" fn sst_partitioner_can_do_trivial_move<P: SstPartitioner>(
ctx: *mut c_void,
smallest_user_key: *const c_char,
smallest_user_key_len: size_t,
largest_user_key: *const c_char,
largest_user_key_len: size_t,
) -> c_uchar {
let partitioner = unsafe { &*(ctx as *mut P) };
let smallest_key =
unsafe { slice::from_raw_parts(smallest_user_key as *const u8, smallest_user_key_len) };
let largest_key =
unsafe { slice::from_raw_parts(largest_user_key as *const u8, largest_user_key_len) };
partitioner.can_do_trivial_move(smallest_key, largest_key) as _
}
pub trait SstPartitionerFactory: Sync + Send {
type Partitioner: SstPartitioner + 'static;
fn name(&self) -> &CString;
fn create_partitioner(&self, context: &SstPartitionerContext) -> Option<Self::Partitioner>;
}
extern "C" fn sst_partitioner_factory_destroy<F: SstPartitionerFactory>(ctx: *mut c_void) {
unsafe {
// Recover from raw pointer and implicitly drop.
Box::from_raw(ctx as *mut F);
}
}
extern "C" fn sst_partitioner_factory_name<F: SstPartitionerFactory>(
ctx: *mut c_void,
) -> *const c_char {
let factory = unsafe { &*(ctx as *mut F) };
factory.name().as_ptr()
}
extern "C" fn sst_partitioner_factory_create_partitioner<F: SstPartitionerFactory>(
ctx: *mut c_void,
context: *mut DBSstPartitionerContext,
) -> *mut DBSstPartitioner {
let factory = unsafe { &*(ctx as *mut F) };
let context = unsafe {
let mut smallest_key_len: usize = 0;
let smallest_key = crocksdb_ffi::crocksdb_sst_partitioner_context_smallest_key(
context,
&mut smallest_key_len,
) as *const u8;
let mut largest_key_len: usize = 0;
let largest_key = crocksdb_ffi::crocksdb_sst_partitioner_context_largest_key(
context,
&mut largest_key_len,
) as *const u8;
SstPartitionerContext {
is_full_compaction: crocksdb_ffi::crocksdb_sst_partitioner_context_is_full_compaction(
context,
) != 0,
is_manual_compaction:
crocksdb_ffi::crocksdb_sst_partitioner_context_is_manual_compaction(context) != 0,
output_level: crocksdb_ffi::crocksdb_sst_partitioner_context_output_level(context),
smallest_key: slice::from_raw_parts(smallest_key, smallest_key_len),
largest_key: slice::from_raw_parts(largest_key, largest_key_len),
}
};
match factory.create_partitioner(&context) {
None => ptr::null_mut(),
Some(partitioner) => {
let ctx = Box::into_raw(Box::new(partitioner)) as *mut c_void;
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_create(
ctx,
sst_partitioner_destructor::<F::Partitioner>,
sst_partitioner_should_partition::<F::Partitioner>,
sst_partitioner_can_do_trivial_move::<F::Partitioner>,
)
}
}
}
}
pub fn new_sst_partitioner_factory<F: SstPartitionerFactory>(
factory: F,
) -> *mut DBSstPartitionerFactory {
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_create(
Box::into_raw(Box::new(factory)) as *mut c_void,
sst_partitioner_factory_destroy::<F>,
sst_partitioner_factory_name::<F>,
sst_partitioner_factory_create_partitioner::<F>,
)
}
}
#[cfg(test)]
mod test {
use std::{
ffi::{CStr, CString},
sync::{Arc, Mutex},
};
use super::*;
struct TestState {
pub call_create_partitioner: usize,
pub call_should_partition: usize,
pub call_can_do_trivial_move: usize,
pub drop_partitioner: usize,
pub drop_factory: usize,
pub should_partition_result: SstPartitionerResult,
pub can_do_trivial_move_result: bool,
pub no_partitioner: bool,
// SstPartitionerRequest fields
pub prev_user_key: Option<Vec<u8>>,
pub current_user_key: Option<Vec<u8>>,
pub current_output_file_size: Option<u64>,
// can_do_trivial_move params
pub trivial_move_smallest_key: Option<Vec<u8>>,
pub trivial_move_largest_key: Option<Vec<u8>>,
// SstPartitionerContext fields
pub is_full_compaction: Option<bool>,
pub is_manual_compaction: Option<bool>,
pub output_level: Option<i32>,
pub smallest_key: Option<Vec<u8>>,
pub largest_key: Option<Vec<u8>>,
}
impl Default for TestState {
fn default() -> Self {
TestState {
call_create_partitioner: 0,
call_should_partition: 0,
call_can_do_trivial_move: 0,
drop_partitioner: 0,
drop_factory: 0,
should_partition_result: SstPartitionerResult::NotRequired,
can_do_trivial_move_result: false,
no_partitioner: false,
prev_user_key: None,
current_user_key: None,
current_output_file_size: None,
trivial_move_smallest_key: None,
trivial_move_largest_key: None,
is_full_compaction: None,
is_manual_compaction: None,
output_level: None,
smallest_key: None,
largest_key: None,
}
}
}
struct TestSstPartitioner {
state: Arc<Mutex<TestState>>,
}
impl SstPartitioner for TestSstPartitioner {
fn should_partition(&self, req: &SstPartitionerRequest) -> SstPartitionerResult {
let mut s = self.state.lock().unwrap();
s.call_should_partition += 1;
s.prev_user_key = Some(req.prev_user_key.to_vec());
s.current_user_key = Some(req.current_user_key.to_vec());
s.current_output_file_size = Some(req.current_output_file_size);
s.should_partition_result
}
fn can_do_trivial_move(&self, smallest_key: &[u8], largest_key: &[u8]) -> bool {
let mut s = self.state.lock().unwrap();
s.call_can_do_trivial_move += 1;
s.trivial_move_smallest_key = Some(smallest_key.to_vec());
s.trivial_move_largest_key = Some(largest_key.to_vec());
s.can_do_trivial_move_result
}
}
impl Drop for TestSstPartitioner {
fn drop(&mut self) {
self.state.lock().unwrap().drop_partitioner += 1;
}
}
lazy_static! {
static ref FACTORY_NAME: CString =
CString::new(b"TestSstPartitionerFactory".to_vec()).unwrap();
}
struct TestSstPartitionerFactory {
state: Arc<Mutex<TestState>>,
}
impl SstPartitionerFactory for TestSstPartitionerFactory {
type Partitioner = TestSstPartitioner;
fn name(&self) -> &CString {
&FACTORY_NAME
}
fn create_partitioner(&self, context: &SstPartitionerContext) -> Option<Self::Partitioner> {
let mut s = self.state.lock().unwrap();
s.call_create_partitioner += 1;
if s.no_partitioner {
return None;
}
s.is_full_compaction = Some(context.is_full_compaction);
s.is_manual_compaction = Some(context.is_manual_compaction);
s.output_level = Some(context.output_level);
s.smallest_key = Some(context.smallest_key.to_vec());
s.largest_key = Some(context.largest_key.to_vec());
Some(TestSstPartitioner {
state: self.state.clone(),
})
}
}
impl Drop for TestSstPartitionerFactory {
fn drop(&mut self) {
self.state.lock().unwrap().drop_factory += 1;
}
}
#[test]
fn factory_name() {
let s = Arc::new(Mutex::new(TestState::default()));
let factory = new_sst_partitioner_factory(TestSstPartitionerFactory { state: s });
let factory_name =
unsafe { CStr::from_ptr(crocksdb_ffi::crocksdb_sst_partitioner_factory_name(factory)) };
assert_eq!(*FACTORY_NAME.as_c_str(), *factory_name);
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_destroy(factory);
}
}
#[test]
fn factory_create_partitioner() {
const IS_FULL_COMPACTION: bool = false;
const IS_MANUAL_COMPACTION: bool = true;
const OUTPUT_LEVEL: i32 = 3;
const SMALLEST_KEY: &[u8] = b"aaaa";
const LARGEST_KEY: &[u8] = b"bbbb";
let s = Arc::new(Mutex::new(TestState::default()));
let factory = new_sst_partitioner_factory(TestSstPartitionerFactory { state: s.clone() });
let context = unsafe { crocksdb_ffi::crocksdb_sst_partitioner_context_create() };
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_context_set_is_full_compaction(
context,
IS_FULL_COMPACTION as _,
);
crocksdb_ffi::crocksdb_sst_partitioner_context_set_is_manual_compaction(
context,
IS_MANUAL_COMPACTION as _,
);
crocksdb_ffi::crocksdb_sst_partitioner_context_set_output_level(context, OUTPUT_LEVEL);
crocksdb_ffi::crocksdb_sst_partitioner_context_set_smallest_key(
context,
SMALLEST_KEY.as_ptr() as *const c_char,
SMALLEST_KEY.len(),
);
crocksdb_ffi::crocksdb_sst_partitioner_context_set_largest_key(
context,
LARGEST_KEY.as_ptr() as *const c_char,
LARGEST_KEY.len(),
);
}
let partitioner = unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_create_partitioner(factory, context)
};
{
let sl = s.lock().unwrap();
assert_eq!(1, sl.call_create_partitioner);
assert_eq!(IS_FULL_COMPACTION, sl.is_full_compaction.unwrap());
assert_eq!(IS_MANUAL_COMPACTION, sl.is_manual_compaction.unwrap());
assert_eq!(OUTPUT_LEVEL, sl.output_level.unwrap());
assert_eq!(SMALLEST_KEY, sl.smallest_key.as_ref().unwrap().as_slice());
assert_eq!(LARGEST_KEY, sl.largest_key.as_ref().unwrap().as_slice());
}
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_destroy(partitioner);
crocksdb_ffi::crocksdb_sst_partitioner_factory_destroy(factory);
}
}
#[test]
fn factory_create_no_partitioner() {
let s = Arc::new(Mutex::new(TestState::default()));
s.lock().unwrap().no_partitioner = true;
let factory = new_sst_partitioner_factory(TestSstPartitionerFactory { state: s.clone() });
let context = unsafe { crocksdb_ffi::crocksdb_sst_partitioner_context_create() };
let partitioner = unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_create_partitioner(factory, context)
};
assert_eq!(1, s.lock().unwrap().call_create_partitioner);
assert_eq!(ptr::null_mut(), partitioner);
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_destroy(factory);
}
}
#[test]
fn partitioner_should_partition() {
const SHOULD_PARTITION: SstPartitionerResult = SstPartitionerResult::Required;
const PREV_KEY: &[u8] = b"test_key_abc";
const CURRENT_KEY: &[u8] = b"test_key_def";
const CURRENT_OUTPUT_FILE_SIZE: u64 = 1234567;
let s = Arc::new(Mutex::new(TestState::default()));
s.lock().unwrap().should_partition_result = SHOULD_PARTITION;
let factory = new_sst_partitioner_factory(TestSstPartitionerFactory { state: s.clone() });
let context = unsafe { crocksdb_ffi::crocksdb_sst_partitioner_context_create() };
let partitioner = unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_create_partitioner(factory, context)
};
let req = unsafe { crocksdb_ffi::crocksdb_sst_partitioner_request_create() };
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_request_set_prev_user_key(
req,
PREV_KEY.as_ptr() as *const c_char,
PREV_KEY.len(),
);
crocksdb_ffi::crocksdb_sst_partitioner_request_set_current_user_key(
req,
CURRENT_KEY.as_ptr() as *const c_char,
CURRENT_KEY.len(),
);
crocksdb_ffi::crocksdb_sst_partitioner_request_set_current_output_file_size(
req,
CURRENT_OUTPUT_FILE_SIZE,
);
}
let should_partition =
unsafe { crocksdb_ffi::crocksdb_sst_partitioner_should_partition(partitioner, req) };
assert_eq!(SHOULD_PARTITION, should_partition);
{
let sl = s.lock().unwrap();
assert_eq!(1, sl.call_create_partitioner);
assert_eq!(1, sl.call_should_partition);
assert_eq!(0, sl.call_can_do_trivial_move);
assert_eq!(PREV_KEY, sl.prev_user_key.as_ref().unwrap().as_slice());
assert_eq!(
CURRENT_KEY,
sl.current_user_key.as_ref().unwrap().as_slice()
);
assert_eq!(
CURRENT_OUTPUT_FILE_SIZE,
sl.current_output_file_size.unwrap()
);
}
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_destroy(partitioner);
crocksdb_ffi::crocksdb_sst_partitioner_factory_destroy(factory);
}
}
#[test]
fn partitioner_can_do_trivial_move() {
const SMALLEST_KEY: &[u8] = b"test_key_abc";
const LARGEST_KEY: &[u8] = b"test_key_def";
const RESULT: bool = true;
let s = Arc::new(Mutex::new(TestState::default()));
s.lock().unwrap().can_do_trivial_move_result = RESULT;
let factory = new_sst_partitioner_factory(TestSstPartitionerFactory { state: s.clone() });
let context = unsafe { crocksdb_ffi::crocksdb_sst_partitioner_context_create() };
let partitioner = unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_create_partitioner(factory, context)
};
let result = unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_can_do_trivial_move(
partitioner,
SMALLEST_KEY.as_ptr() as *const c_char,
SMALLEST_KEY.len(),
LARGEST_KEY.as_ptr() as *const c_char,
LARGEST_KEY.len(),
)
};
{
let sl = s.lock().unwrap();
assert_eq!(1, sl.call_create_partitioner);
assert_eq!(0, sl.call_should_partition);
assert_eq!(1, sl.call_can_do_trivial_move);
assert_eq!(
SMALLEST_KEY,
sl.trivial_move_smallest_key.as_ref().unwrap().as_slice()
);
assert_eq!(
LARGEST_KEY,
sl.trivial_move_largest_key.as_ref().unwrap().as_slice()
);
assert_eq!(RESULT, result);
}
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_destroy(partitioner);
crocksdb_ffi::crocksdb_sst_partitioner_factory_destroy(factory);
}
}
#[test]
fn drop() {
let s = Arc::new(Mutex::new(TestState::default()));
let factory = new_sst_partitioner_factory(TestSstPartitionerFactory { state: s.clone() });
let context = unsafe { crocksdb_ffi::crocksdb_sst_partitioner_context_create() };
let partitioner = unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_create_partitioner(factory, context)
};
{
let sl = s.lock().unwrap();
assert_eq!(0, sl.drop_partitioner);
assert_eq!(0, sl.drop_factory);
}
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_destroy(partitioner);
}
{
let sl = s.lock().unwrap();
assert_eq!(1, sl.drop_partitioner);
assert_eq!(0, sl.drop_factory);
}
unsafe {
crocksdb_ffi::crocksdb_sst_partitioner_factory_destroy(factory);
}
{
let sl = s.lock().unwrap();
assert_eq!(1, sl.drop_partitioner);
assert_eq!(1, sl.drop_factory);
}
}
}
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment