TiKV Region Split 全流程分析

2022-05-26

分裂可以说是 Region 生命周期中最为重要的一步,如同细胞一般,分裂是 Region 被创造并持续增多的唯一方式。

本文将介绍以下内容:

  • Region Split 是由谁触发的。
  • Region Split 是如何计算 Split Key 的。
  • Region Split 最终是如何执行的。

我们先来看一个 Region Split 过程的大致流程:

  1. TiKV/PD/TiDB 触发 Region Split 事件。
  2. Raftstore 处理 Region Split 事件,计算 Split Key。
  3. Raftstore 执行 Split。

Region Split 的触发方式

我们可以将 Region 的分裂从动机上分为两类:

  • 内部机制导致的 Region 被动分裂(例如 Region 的大小超过阈值,Load Base Split 被触发等)
  • 人工手段对 Region 进行主动分裂(建表或手动 Split Region)

TiKV 触发分裂

因为 Region 是 TiKV 的逻辑存储单元,Region 最基本的分裂方式也是来源于 TiKV 的控制。

定期检查

TiKV 默认会 10s 进行一次 Region 的分裂检查,此举由 Raft 状态机驱动,定期 Tick 进行触发。函数名称为 PeerFsmDelegate::on_split_region_check_tick

因为 Region Split 的行为后续会作为一条 Raft log 在副本间进行同步,所以该函数会首先检查当前 Region peer 是否为 leader,以避免进行无用的检查。

if !self.fsm.peer.is_leader() {
    return;
}

if self.fsm.peer.may_skip_split_check
    && self.fsm.peer.compaction_declined_bytes < self.ctx.cfg.region_split_check_diff().0
    && self.fsm.peer.size_diff_hint < self.ctx.cfg.region_split_check_diff().0
{
    return;
}

紧接着 Leader check 之后,就是对 Split 必要性的检查,为了避免过多的 Split check,我们设置了以下 3 个条件来进行过滤:

  • Region peer 的 may_skip_split_check flag 是否为 True
  • Region peer 的 compaction_declined_bytes 是否小于 region-split-check-diff 阈值
  • Region peer 的 size_diff_hint 是否小于 region-split-check-diff 阈值

may_skip_split_check 的 flag 会在必要时被设置为 False 来确保 Split 检查会尽可能地被执行(例如 TiKV 刚刚启动时)。compaction_declined_bytessize_diff_hint 均是对 Region 大小变化的增量统计(分别统计自 Compaction 数据和 Apply 数据的过程),它们在此隐含了这样一个条件:只有 Region 的大小变化超过 region-split-check-diff 后才需要进行分裂检查(这个配置的默认值是 region-split-size 的 1/16,即 96 / 16 = 6 MB)。

而后就是一些特殊逻辑的检查,在此不进一步展开,他们包括:

  • 当前是否有堆积未完成的 Split 任务
  • 当前是否处于 Lightning/BR 的导入过程中
  • 当前是否正在生成 Snapshot

需要注意此阶段的检查仅仅是触发了 Region Split 的事件,具体能否分裂以及如何分裂还取决于后续的 Split 触发过程。

Load Base Split

TiKV 还有一个会触发 Region Split 的功能来自于 Load Base Split。其核心代码位于 AutoSplitController::flush。StatsMonitor 会收集读请求的统计信息,包括请求的数目,请求读取的流量以及读取的 Key Range 等。对于 QPS 或 Byte 满足 qps_thresholdbyte_threshold 的 Region,则会在之前收集的 Key Range 基础上对 Key 进行采样,选择一个切分后左右 Region 上的请求数量最为均衡的 Key 作为切分点进行切分。

PD 触发分裂

PD 也可以进行分裂的触发。此举可以通过以下方式进行:

  • 调用 /regions/split 的 HTTP API 触发
  • 通过 pd-ctl 创建 Operator 触发
  • 通过调用 gRPC 接口 SplitRegions/SplitAndScatterRegions 来触发

其中,pd-ctl 作为主要面向用户的操作,方式如下:

>> operator add split-region 1 --policy=approximate     //  Region 1 对半拆分成两个 Region,基于粗略估计值
>> operator add split-region 1 --policy=scan            //  Region 1 对半拆分成两个 Region,基于精确扫描值

上述操作的本质都是创建一个 Split 的 Operator 并下发给对应 Region。具体的 PD 侧代码可以通过 RegionSplitter::SplitRegions 函数进行自上而下的研究,在此不多做表述。

Operator 通过 Region 心跳下发给 TiKV 后,TiKV 会根据下发的 Split 任务类型去创建对应的事件,具体代码在此

if resp.has_split_region() {
    let mut split_region = resp.take_split_region();
    info!("try to split"; "region_id" => region_id, "region_epoch" => ?epoch);
    let msg = if split_region.get_policy() == pdpb::CheckPolicy::Usekey {
        CasualMessage::SplitRegion {
            region_epoch: epoch,
            split_keys: split_region.take_keys().into(),
            callback: Callback::None,
            source: "pd".into(),
        }
    } else {
        CasualMessage::HalfSplitRegion {
            region_epoch: epoch,
            policy: split_region.get_policy(),
            source: "pd",
            cb: Callback::None,
        }
    };
    if let Err(e) = router.send(region_id, PeerMsg::CasualMessage(msg)) {
        error!("send halfsplit request failed"; "region_id" => region_id, "err" => ?e);
    }
}

可以看到根据不同的 Split 方式,所创建的事件也不同——若是给定了分裂点 Key 则会直接下发 CasualMessage::SplitRegion 事件,否则根据不同的分裂策略创建一个 CasualMessage::HalfSplitRegion 事件,期以对 Region 进行对半分。这里的策略主要分为 Scan 和 Approximate 两类,具体的区别会在后文中进行介绍。

TiDB 触发分裂

DDL

在建表或添加分区时,TiDB 会在 DDL 阶段对表的 Region 进行预切分,为每个表或分区创建单独的 Region,用于避免发生大量建表和写入造成的热点问题。此举也是通过调用 PD 的 Split 接口达成的(早期版本是 TiDB 直接下发给 TiKV,现已废弃)。具体的代码入口在 ddl::preSplitAndScatter 接口,你可以通过该方法的调用情况来看不同的 Split Table 发生在何时何处。

SQL

除了建表时自动为每个表切分出的一个 Region,如果在单表内部存在写入热点,我们也可以通过 SQL 来手动 Split Region。这个原理其实和上述的 DDL 过程相同,均是调用统一的 SplitRegions 接口来进行 Split 任务的下发。

具体的 SQL 语法可以参考官方文档:Split Region 使用文档

其他

上面只阐述了 3 大组件的常见 Region Split 触发流程,事实上还有很多其他机制会触发 Region Split,例如 Lightning/BR 这样的工具导入数据前也会对 Region 进行预切分和打散,以求导入后数据的均衡。tikv-ctl 也可以触发 Region 的 Split。

Region Split Key 的计算方式

以上述方式触发 Region Split 事件后,具体的 Split 的 Key 可以以多种方式和维度被计算出来。例如通过精确的 Scan 扫描来确定 Region 大小上的中点进行分裂,或通过指定的 Key 直接进行分裂等,不同的方式往往用于不同的场景,具体原理如下。

Coprocessor

此 Coprocessor 非 TiKV 中用于下推 SQL 执行的 Coprocessor,而是 raftstore 代码中的一个概念。其主要作用相当于外挂在 TiKV 的 Raft 层上的一个协处理工具集合,用于观测和处理与 Raft 相关的周边事件。SplitChecker 就是其中之一,用于接受,处理和下发与 Region Split 有关的事件。

/// SplitChecker is invoked during a split check scan, and decides to use
/// which keys to split a region.
pub trait SplitChecker<E> {
    /// Hook to call for every kv scanned during split.
    ///
    /// Return true to abort scan early.
    fn on_kv(&mut self, _: &mut ObserverContext<'_>, _: &KeyEntry) -> bool {
        false
    }

    /// Get the desired split keys.
    fn split_keys(&mut self) -> Vec<Vec<u8>>;

    /// Get approximate split keys without scan.
    fn approximate_split_keys(&mut self, _: &Region, _: &E) -> Result<Vec<Vec<u8>>> {
        Ok(vec![])
    }

    /// Get split policy.
    fn policy(&self) -> CheckPolicy;
}

一个 SplitChecker 包含 4 个方法,分别是:

  • on_kv,在使用 Scan 方式时,用于在 Iterator 扫描 Key 的过程中接受 Key,并在内部维护对应的状态来实现不同的分裂方式。
  • split_keys,在完成扫描后通过此方法来拿到最终的 Split Key 结果。
  • approximate_split_keys,在使用 Approximate 方式时,不进行 Scan 而直接拿到 Split Key 结果
  • policy,返回当前的 Split 检查策略,有 Scan/Approximate 两种方式。

对这 4 个方法不同的实现也就决定了不同的分裂方式,下面我们分别介绍 TiKV 内部支持的所有不同的分裂方式。

Half

HalfCheckObserver 实现了对 Region 的 Sizie 对半切策略,在 Scan 模式下,为了找到一个 Region 内 Size 维度上的中点,把所有的 Key 都记录下来显然是不合理的,这样可能会占用大量的内存。取而代之的方式是根据配置计算出一个最小的 Size 单位 n MB,计算函数名为 half_split_bucket_size 通过将 region_max_size 除以 BUCKET_NUMBER_LIMIT(常量,值为 1024),计算出一个 Bucket 大小,最小为 1 MB,最大为 512 MB。

fn half_split_bucket_size(region_max_size: u64) -> u64 {
    let mut half_split_bucket_size = region_max_size / BUCKET_NUMBER_LIMIT as u64;
    let bucket_size_limit = ReadableSize::mb(BUCKET_SIZE_LIMIT_MB).0;
    if half_split_bucket_size == 0 {
        half_split_bucket_size = 1;
    } else if half_split_bucket_size > bucket_size_limit {
        half_split_bucket_size = bucket_size_limit;
    }
    half_split_bucket_size
}

在后续的扫描过程中,仅在每扫描过 n MB 大小后才记录下当前的 Key,这样可以通过牺牲一定的精度换来了较少的内存占用。

fn on_kv(&mut self, _: &mut ObserverContext<'_>, entry: &KeyEntry) -> bool {
    if self.buckets.is_empty() || self.cur_bucket_size >= self.each_bucket_size {
        self.buckets.push(entry.key().to_vec());
        self.cur_bucket_size = 0;
    }
    self.cur_bucket_size += entry.entry_size() as u64;
    false
}

fn split_keys(&mut self) -> Vec<Vec<u8>> {
    let mid = self.buckets.len() / 2;
    if mid == 0 {
        vec![]
    } else {
        let data_key = self.buckets.swap_remove(mid);
        let key = keys::origin_key(&data_key).to_vec();
        vec![key]
    }
}

在后续计算中点 Key 的过程中,也只需要取我们收集到的 Key 的中间元素,即可获得近似的 Region Size 中点,用于后续的切分。

对于具体 approximate_split_keys 的实现取决于不同的 KV Engine,以默认的 RocksDB 为例,为了避免对整个区间上全 Key-Value 的扫描,我们使用了 RocksDB 的 TableProperties 特性,来在 RocksDB 构建每个 SST 文件的时候就提前收集一些 Key 相关的信息,从而可以在此时避免进行 I/O 操作即可获得近似的 Key Range 上的 Key 信息,再辅之以采样等手段,相较于 Scan 策略会更不精准,但省去了不少资源。对应的代码在 RocksEngine::get_range_approximate_split_keys_cf 方法中。

Size

SizeCheckObserver 实现了根据 Region Size 切分 Region 的策略。其逻辑相对简单,在默认配置下,会对 Region 的 KV 进行 Scan 遍历,每扫描过 96 MB 的数据便会记录下当前的 Key,一次最多记录 10 个。

fn on_kv(&mut self, _: &mut ObserverContext<'_>, entry: &KeyEntry) -> bool {
    let size = entry.entry_size() as u64;
    self.current_size += size;

    let mut over_limit = self.split_keys.len() as u64 >= self.batch_split_limit;
    if self.current_size > self.split_size && !over_limit {
        self.split_keys.push(keys::origin_key(entry.key()).to_vec());
        // if for previous on_kv() self.current_size == self.split_size,
        // the split key would be pushed this time, but the entry size for this time should not be ignored.
        self.current_size = if self.current_size - size == self.split_size {
            size
        } else {
            0
        };
        over_limit = self.split_keys.len() as u64 >= self.batch_split_limit;
    }

    // For a large region, scan over the range maybe cost too much time,
    // so limit the number of produced split_key for one batch.
    // Also need to scan over self.max_size for last part.
    over_limit && self.current_size + self.split_size >= self.max_size
}

fn split_keys(&mut self) -> Vec<Vec<u8>> {
    // make sure not to split when less than max_size for last part
    if self.current_size + self.split_size < self.max_size {
        self.split_keys.pop();
    }
    if !self.split_keys.is_empty() {
        std::mem::take(&mut self.split_keys)
    } else {
        vec![]
    }
}

approximate_split_keys 的实现和 Half 类似,在此不表,依然是基于 RocksDB 的 TableProperties 功能。

Keys

KeysCheckObserver 实现了根据 Region Key 数量切分 Region 的策略,其原理和 SizeCheckObserver 相同,只不过把计算方式改成了 Key 数量的统计,在此不过多展开,

Tabel

TableCheckObserver 实现了根据 Region 范围内 Key 所属的 Table 进行切分的策略。这个 Checker 的实现比较特殊,它在 TiKV 内部引入了 SQL 层的概念。原理也比较简单,在 Scan 时去 Decode 每个 Key,检查其所属的表 ID 和之前 Key 是否相同,若不同则加入 Split Key 进行分裂。

/// Feed keys in order to find the split key.
/// If `current_data_key` does not belong to `status.first_encoded_table_prefix`.
/// it returns the encoded table prefix of `current_data_key`.
fn on_kv(&mut self, _: &mut ObserverContext<'_>, entry: &KeyEntry) -> bool {
    if self.split_key.is_some() {
        return true;
    }

    let current_encoded_key = keys::origin_key(entry.key());

    let split_key = if self.first_encoded_table_prefix.is_some() {
        if !is_same_table(
            self.first_encoded_table_prefix.as_ref().unwrap(),
            current_encoded_key,
        ) {
            // Different tables.
            Some(current_encoded_key)
        } else {
            None
        }
    } else if is_table_key(current_encoded_key) {
        // Now we meet the very first table key of this region.
        Some(current_encoded_key)
    } else {
        None
    };
    self.split_key = split_key.and_then(to_encoded_table_prefix);
    self.split_key.is_some()
}

由于工作原理决定了它只能基于 Scan 策略进行工作,所以没有提供 approximate_split_keys 方法的实现。

优先级

上面一共介绍了 TiKV 支持的 4 种 Split 方式,那么具体工作过程中,实际到底哪一个方式会被触发呢?答案是都有可能。

每个 SplitChecker 都会被加入到一个 SplitCheckerHost 中,并被赋予不同的优先级,每次 Split 都会依次“询问”每个 SplitChecker 的“意见”,如果高优先级的 Checker 不能给出 Split Key 那么就依次向更低优先级的 Checker 轮训,直到得到一个 Split Key 或确认无法 Split。优先级在将 SplitChecker 注册到 Coprocessor 时就被定义好了,代码位于 CoprocessorHost::new

pub fn new<C: CasualRouter<E> + Clone + Send + 'static>(
    ch: C,
    cfg: Config,
) -> CoprocessorHost<E> {
    let mut registry = Registry::default();
    registry.register_split_check_observer(
        200,
        BoxSplitCheckObserver::new(SizeCheckObserver::new(ch.clone())),
    );
    registry.register_split_check_observer(
        200,
        BoxSplitCheckObserver::new(KeysCheckObserver::new(ch)),
    );
    registry.register_split_check_observer(100, BoxSplitCheckObserver::new(HalfCheckObserver));
    registry.register_split_check_observer(
        400,
        BoxSplitCheckObserver::new(TableCheckObserver::default()),
    );
    CoprocessorHost { registry, cfg }
}

可以看到 HalfCheckObserver 有最高优先级,其次是 SizeCheckObserverKeysCheckObserverTableCheckObserver 最低。但是我们所见到的大多数 Region 分裂都是基于 Size 的,Half 分裂尽管有最高优先级,为什么不会被频繁触发呢?答案是我们每次基于注册在 Coprocessor 的 Split Checker 创建 SplitCheckerHost 时(代码入口在 CoprocessorHost::new_split_checker_host),并不会将所有的 Checker 都导入,而是根据不同的配置以及场景进行有选择的添加。例如只有 auto_split 选项设置为关闭时,HalfCheckObserver 才会被添加到 Host 中,这个选项在 TiKV 定时检查触发 Split 时会开启,所以在对应场景下 HalfCheckObserver 不会起作用。

#[derive(Clone)]
pub struct HalfCheckObserver;

impl Coprocessor for HalfCheckObserver {}

impl<E> SplitCheckObserver<E> for HalfCheckObserver
where
    E: KvEngine,
{
    fn add_checker(
        &self,
        _: &mut ObserverContext<'_>,
        host: &mut Host<'_, E>,
        _: &E,
        policy: CheckPolicy,
    ) {
        if host.auto_split() {
            return;
        }
        host.add_checker(Box::new(Checker::new(
            half_split_bucket_size(host.cfg.region_max_size().0),
            policy,
        )))
    }
}

再例如只有当 split_region_on_table 配置开启时,TableCheckObserver 才会被添加到 Host 中,该配置默认关闭。

#[derive(Default, Clone)]
pub struct TableCheckObserver;

impl Coprocessor for TableCheckObserver {}

impl<E> SplitCheckObserver<E> for TableCheckObserver
where
    E: KvEngine,
{
    fn add_checker(
        &self,
        ctx: &mut ObserverContext<'_>,
        host: &mut Host<'_, E>,
        engine: &E,
        policy: CheckPolicy,
    ) {
        if !host.cfg.split_region_on_table {
            return;
        }
        ...
}

所以说在大多数情况下,只有 KeysCheckObserverSizeCheckObserver 主导 Region 的分裂方式。

Region Split 的执行过程

通过 Raftstore 的 Coprocessor 确定好 Region 的 Split Key 后,最后就来到了 Split 的执行阶段。Region 的 Split 任务会被下发到具体的 Region,继而触发 PeerFsmDelegate::on_prepare_split_region 函数,正式开启 Region 的 Split 执行。

Pre-check

首先 TiKV 会再次确认当前 Region 为 leader,并检查 Epoch 等属性是否发生了变化,Epoch 内的 Version 属性只有在完成 Split 或 Merge 的情况下才会增加,因为 Version 一定是严格单调递增的,所以 PD 使用了这个规则去判断范围重叠的不同 Region 的新旧。在检查通过后,便向 PD 发送 AskBatchSplit 请求为即将分裂出来的新 Region 获取 Region ID,并触发 Raft 开始进行 Split log 的 Proposal。

info!(
    "try to batch split region";
    "region_id" => region.get_id(),
    "new_region_ids" => ?resp.get_ids(),
    "region" => ?region,
    "task" => task,
);

let req = new_batch_split_region_request(
    split_keys,
    resp.take_ids().into(),
    right_derive,
);
let region_id = region.get_id();
let epoch = region.take_region_epoch();
send_admin_request(
    &router,
    region_id,
    epoch,
    peer,
    req,
    callback,
    Default::default(),
);

Raft Proposal & Apply

通过 Raft log 将 Split 同步到各个 Peer 之上完成 Commit 之后,ApplyDelegate::exec_batch_split 便开始执行 Region 的分裂。创建新 Region,更改 Region 边界,并将 Region 的新信息写入落盘。

for new_region in &regions {
    if new_region.get_id() == derived.get_id() {
        continue;
    }
    let new_split_peer = new_split_regions.get(&new_region.get_id()).unwrap();
    if let Some(ref r) = new_split_peer.result {
        warn!(
            "new region from splitting already exists";
            "new_region_id" => new_region.get_id(),
            "new_peer_id" => new_split_peer.peer_id,
            "reason" => r,
            "region_id" => self.region_id(),
            "peer_id" => self.id(),
        );
        continue;
    }
    write_peer_state(kv_wb_mut, new_region, PeerState::Normal, None)
        .and_then(|_| write_initial_apply_state(kv_wb_mut, new_region.get_id()))
        .unwrap_or_else(|e| {
            panic!(
                "{} fails to save split region {:?}: {:?}",
                self.tag, new_region, e
            )
        });
}
write_peer_state(kv_wb_mut, &derived, PeerState::Normal, None).unwrap_or_else(|e| {
    panic!("{} fails to update region {:?}: {:?}", self.tag, derived, e)
});

在默认的分裂方式下,原 Region 要分裂到右侧,举例而言,假设分裂前的 Region 数量一共有 2 个,ID 分别为 1 和 2。2 是即将要分裂的 Region,且 Split Key 为 "b"。

Region 1 ["", "a"), Region 2 ["a", "")

分裂后的新 Region 被分配了 ID 3,那么分裂后的 Region 会形如:

Region 1 ["", "a"), Region 3 ["a", "b"), Region 2 ["b", "")

在 TiKV 完成 Split log 的 Apply 后,会通过 ApplyResult::Res 事件触发 PeerFsmDelegate::on_ready_split_region 来完成 Split 的预后工作。如果当前 Region 是 leader,则会给 PD 发送一个 Report(Batch)Split 的 RPC 请求,仅供 PD 打个日志记录,方便我们在查问题时通过 PD 的日志看到各个 Region 的 Split 记录。由于 Region 的 ID 分配也是严格保证单调递增,所以我们可以说 Region ID 越大的 Region 则越新。

if is_leader {
    self.fsm.peer.approximate_size = estimated_size;
    self.fsm.peer.approximate_keys = estimated_keys;
    self.fsm.peer.heartbeat_pd(self.ctx);
    // Notify pd immediately to let it update the region meta.
    info!(
        "notify pd with split";
        "region_id" => self.fsm.region_id(),
        "peer_id" => self.fsm.peer_id(),
        "split_count" => regions.len(),
    );
    // Now pd only uses ReportBatchSplit for history operation show,
    // so we send it independently here.
    let task = PdTask::ReportBatchSplit {
        regions: regions.to_vec(),
    };
    if let Err(e) = self.ctx.pd_scheduler.schedule(task) {
        error!(
            "failed to notify pd";
            "region_id" => self.fsm.region_id(),
            "peer_id" => self.fsm.peer_id(),
            "err" => %e,
        );
    }
}

其余则是一些向 PD 上报心跳,统计信息的初始化工作,更新分裂后的 Region epoch 并在 Raft group 中注册 Region 的路由。这些工作完成后,当前 TiKV 上的 Region 可以说是已经完成分裂了。

Raft Election

对于分裂前的原 Region 是 Leader 的 Peer 来说,分裂后的 Region 是可以立马发起选举的,而对于原 Region 非 Leader 的 Peer 来说,它分裂创建出的新 Region 是不能立马发起选举的,而是需要等待一个 Raft 的选举超时时间。这样实现的原因是存在下列的 Case:

  1. 假设有一个 3 副本的 Region
  2. Split 的 Log 已经复制到了所有的 Follower 上
  3. 所有的 Follower 完成了 Region Split Log 的 Apply,完成了分裂
  4. Region 的 Leader 还没有开始或完成分裂

如果允许原 Peer 非 Leader 的新 Region 分裂出来后立马开始选举,则会出现同一个数据范围内存在两个 Region leader 对外提供服务,一个是分裂后的新的更小的 Region leader,一个是尚未分裂的原 Region leader(Lease 尚未过期),这样一来就存在破坏线性一致性的可能。由于一次 Raft 的选举超时时间要大于 Leader 的 Lease 时间,所以只要我们保证以下两点:

  1. 完成分裂的 Region 等待一个 Raft 的选举超时时间再开始选举
  2. 需要 Split 的 Region 不再续约 Lease

所以当新分裂的 Region 开始选举时,旧的 Region leader 早些时候一定会因为发现自身的 Epoch 与其余两个 Follower 不同而选举失败完成退选。

踩坑经验

Split Key 的格式为 Encoded Key without TS

在 TiDB 和 TiKV 的语境下,当我们说到 Key 编码时,它可能指的是以下几种情况:

  • Raw Key
  • Encoded Key without TS
  • Encoded Key with TS

TiDB 在发送请求时使用的是 Raw Key,也即不带任何与 MVCC 相关的信息,也没有 Padding,只包括诸如 TableID,RowID 等基本信息。

TiKV 的 Raftstore 以及 PD 在处理诸如 Region 边界,Split 等 Key 时使用的是 Encoded Key without TS,它在 Raw Key 的基础上进行了 Encode,添加了用于保持字典序的 Padding,但由于此层尚未涉及到具体的事务,所以并没有 TS 参与其中。

TiKV 在实际读写底层 RocksDB 数据时,会将请求的 TS 一并 Encode 到 Key 里来区分 MVCC 信息,所以这一层使用的是 Encoded Key with TS。

Region Split 发生在 Raftstore 这一层,所以其格式均为 Encoded Key without TS,在开发相关功能时,要注意对 Key 进行 Encode,并且剔除 TS 信息,以免出现一些预期外的行为。

参考

Tagged in : TiKV Region Raft Split