1
  2
  3
  4
  5
  6
  7
  8
  9
 10
 11
 12
 13
 14
 15
 16
 17
 18
 19
 20
 21
 22
 23
 24
 25
 26
 27
 28
 29
 30
 31
 32
 33
 34
 35
 36
 37
 38
 39
 40
 41
 42
 43
 44
 45
 46
 47
 48
 49
 50
 51
 52
 53
 54
 55
 56
 57
 58
 59
 60
 61
 62
 63
 64
 65
 66
 67
 68
 69
 70
 71
 72
 73
 74
 75
 76
 77
 78
 79
 80
 81
 82
 83
 84
 85
 86
 87
 88
 89
 90
 91
 92
 93
 94
 95
 96
 97
 98
 99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
// 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(&mut self, req: &SstPartitionerRequest) -> SstPartitionerResult;
    fn can_do_trivial_move(&mut 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 { &mut *(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 { &mut *(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(&mut 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(&mut 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);
        }
    }
}