mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-05-17 13:30:38 +00:00
refactor(mito2)!: remove PartitionTreeMemtable (#8080)
* feat: switch partition tree to bulk Signed-off-by: evenyag <realevenyag@gmail.com> * chore: keep partition tree memtable for migration test Restore PartitionTreeMemtable construction when memtable.type=partition_tree is explicit, and move the sparse-encoding bulk override into the default (no explicit memtable.type) arm so phase 2's memtable.type=bulk wins on reopen. Rewrite test_reopen_time_series_sparse_memtable_with_bulk to use a metric-engine-shaped schema and sparse-encoded rows with WriteHint::Sparse, so the test actually exercises a PartitionTreeMemtable in phase 1 and verifies WAL replay into the new BulkMemtable on reopen without flushing. Signed-off-by: evenyag <realevenyag@gmail.com> * chore: drop partition tree memtable from runtime Re-apply the unconditional sparse-encoding override in `MemtableBuilderProvider::builder_for_options` and route the `MemtableOptions::PartitionTree` arm to `BulkMemtable` with a deprecation warning. After this change, `PartitionTreeMemtableBuilder` is no longer reachable from the engine runtime; benchmarks still reference the type. Remove `test_reopen_time_series_sparse_memtable_with_bulk` and the `put_sparse_rows` helper added in the previous commit — that test only existed to validate the PartitionTree -> Bulk reopen migration and is unnecessary now that the override is in place. Signed-off-by: evenyag <realevenyag@gmail.com> * refactor(mito2): move timestamp_array_to_i64_slice into read module Relocate the timestamp_array_to_i64_slice helper from memtable/partition_tree/data.rs to the read module so that the read path no longer depends on the partition_tree internals. All call sites (both inside and outside the partition_tree module) now import from crate::read. Signed-off-by: evenyag <realevenyag@gmail.com> * refactor(mito2): use TimeSeriesMemtableBuilder in time_partition tests The time_partition tests use the memtable builder purely as a generic backend for the TimePartitions write/scan paths; nothing in them is specific to the partition-tree memtable. Switch the seven affected tests to TimeSeriesMemtableBuilder so the tests no longer depend on PartitionTreeMemtableBuilder. Signed-off-by: evenyag <realevenyag@gmail.com> * chore(mito2): delete PartitionTreeMemtable implementation The runtime already falls back to BulkMemtable for the PartitionTree variant. Drop the now-unreachable implementation, its metrics, the partition_tree benchmarks, the metric-engine Unsupported fallback in bulk_insert.rs, and the test helpers that only existed for the deleted module. MemtableOptions::PartitionTree, its parsing, the runtime fallback, the store-api MEMTABLE_PARTITION_TREE_* constants, and the SQL fixtures remain so existing region options keep round-tripping. Signed-off-by: evenyag <realevenyag@gmail.com> * refactor(mito-codec): drop skip_partition_column parameter PartitionTreeMemtable was the only caller passing skip_partition_column=true; every other caller passes false. Now that the partition_tree module is gone, the parameter is uniformly false and the guard branch is dead. Drop the parameter from the trait method and both impls, remove the guard and the is_partition_column helper, and update the four remaining call sites in mito2 plus the bench. Signed-off-by: evenyag <realevenyag@gmail.com> * chore(mito2): remove unused MemtableConfig enum Signed-off-by: evenyag <realevenyag@gmail.com> * chore: fmt code Signed-off-by: evenyag <realevenyag@gmail.com> * refactor: remove unused variant Signed-off-by: evenyag <realevenyag@gmail.com> * test: update test_config_api Signed-off-by: evenyag <realevenyag@gmail.com> * fix: remove unused memtable test helpers Signed-off-by: evenyag <realevenyag@gmail.com> * chore: address review comment Signed-off-by: evenyag <realevenyag@gmail.com> * fix: support bulk memtable options Signed-off-by: evenyag <realevenyag@gmail.com> * fix: sanitize config Signed-off-by: evenyag <realevenyag@gmail.com> * feat: remove partition tree options from region options Move primary_key_encoding to the top level Signed-off-by: evenyag <realevenyag@gmail.com> * test: make ssts test datetime replaced text stable Signed-off-by: evenyag <realevenyag@gmail.com> * test: update sqlness result Signed-off-by: evenyag <realevenyag@gmail.com> * chore: validate_enum_options consider bulk memtable Signed-off-by: evenyag <realevenyag@gmail.com> * refactor: pass region id when parsing region options Replace the `TryFrom<&HashMap>` impl for `RegionOptions` with `try_from_options(region_id, options_map)` so the legacy partition_tree fallback can log the affected region. The fallback now also overrides the SST format to flat in addition to clearing the memtable type. Signed-off-by: evenyag <realevenyag@gmail.com> * fix: align sst_format with bulk memtable on parse and open Signed-off-by: evenyag <realevenyag@gmail.com> --------- Signed-off-by: evenyag <realevenyag@gmail.com>
This commit is contained in:
@@ -193,11 +193,6 @@
|
||||
| `region_engine.mito.bloom_filter_index.create_on_compaction` | String | `auto` | Whether to create the bloom filter on compaction.<br/>- `auto`: automatically (default)<br/>- `disable`: never |
|
||||
| `region_engine.mito.bloom_filter_index.apply_on_query` | String | `auto` | Whether to apply the bloom filter on query<br/>- `auto`: automatically (default)<br/>- `disable`: never |
|
||||
| `region_engine.mito.bloom_filter_index.mem_threshold_on_create` | String | `auto` | Memory threshold for bloom filter creation.<br/>- `auto`: automatically determine the threshold based on the system memory size (default)<br/>- `unlimited`: no memory limit<br/>- `[size]` e.g. `64MB`: fixed memory threshold |
|
||||
| `region_engine.mito.memtable` | -- | -- | -- |
|
||||
| `region_engine.mito.memtable.type` | String | `time_series` | Memtable type.<br/>- `time_series`: time-series memtable<br/>- `partition_tree`: partition tree memtable (experimental) |
|
||||
| `region_engine.mito.memtable.index_max_keys_per_shard` | Integer | `8192` | The max number of keys in one shard.<br/>Only available for `partition_tree` memtable. |
|
||||
| `region_engine.mito.memtable.data_freeze_threshold` | Integer | `32768` | The max rows of data inside the actively writing buffer in one shard.<br/>Only available for `partition_tree` memtable. |
|
||||
| `region_engine.mito.memtable.fork_dictionary_bytes` | String | `1GiB` | Max dictionary bytes.<br/>Only available for `partition_tree` memtable. |
|
||||
| `region_engine.file` | -- | -- | Enable the file engine. |
|
||||
| `region_engine.metric` | -- | -- | Metric engine options. |
|
||||
| `region_engine.metric.sparse_primary_key_encoding` | Bool | `true` | Whether to use sparse primary key encoding. |
|
||||
@@ -585,11 +580,6 @@
|
||||
| `region_engine.mito.bloom_filter_index.create_on_compaction` | String | `auto` | Whether to create the index on compaction.<br/>- `auto`: automatically (default)<br/>- `disable`: never |
|
||||
| `region_engine.mito.bloom_filter_index.apply_on_query` | String | `auto` | Whether to apply the index on query<br/>- `auto`: automatically (default)<br/>- `disable`: never |
|
||||
| `region_engine.mito.bloom_filter_index.mem_threshold_on_create` | String | `auto` | Memory threshold for the index creation.<br/>- `auto`: automatically determine the threshold based on the system memory size (default)<br/>- `unlimited`: no memory limit<br/>- `[size]` e.g. `64MB`: fixed memory threshold |
|
||||
| `region_engine.mito.memtable` | -- | -- | -- |
|
||||
| `region_engine.mito.memtable.type` | String | `time_series` | Memtable type.<br/>- `time_series`: time-series memtable<br/>- `partition_tree`: partition tree memtable (experimental) |
|
||||
| `region_engine.mito.memtable.index_max_keys_per_shard` | Integer | `8192` | The max number of keys in one shard.<br/>Only available for `partition_tree` memtable. |
|
||||
| `region_engine.mito.memtable.data_freeze_threshold` | Integer | `32768` | The max rows of data inside the actively writing buffer in one shard.<br/>Only available for `partition_tree` memtable. |
|
||||
| `region_engine.mito.memtable.fork_dictionary_bytes` | String | `1GiB` | Max dictionary bytes.<br/>Only available for `partition_tree` memtable. |
|
||||
| `region_engine.mito.gc` | -- | -- | -- |
|
||||
| `region_engine.mito.gc.enable` | Bool | `false` | Whether GC is enabled. Need to be the same with metasrv's `gc.enable` or unexpected behavior will occur |
|
||||
| `region_engine.mito.gc.lingering_time` | String | `1m` | Lingering time before deleting files.<br/>Should be long enough to allow long running queries to finish.<br/>If set to None, then unused files will be deleted immediately. |
|
||||
|
||||
@@ -646,24 +646,6 @@ apply_on_query = "auto"
|
||||
## - `[size]` e.g. `64MB`: fixed memory threshold
|
||||
mem_threshold_on_create = "auto"
|
||||
|
||||
[region_engine.mito.memtable]
|
||||
## Memtable type.
|
||||
## - `time_series`: time-series memtable
|
||||
## - `partition_tree`: partition tree memtable (experimental)
|
||||
type = "time_series"
|
||||
|
||||
## The max number of keys in one shard.
|
||||
## Only available for `partition_tree` memtable.
|
||||
index_max_keys_per_shard = 8192
|
||||
|
||||
## The max rows of data inside the actively writing buffer in one shard.
|
||||
## Only available for `partition_tree` memtable.
|
||||
data_freeze_threshold = 32768
|
||||
|
||||
## Max dictionary bytes.
|
||||
## Only available for `partition_tree` memtable.
|
||||
fork_dictionary_bytes = "1GiB"
|
||||
|
||||
[region_engine.mito.gc]
|
||||
## Whether GC is enabled. Need to be the same with metasrv's `gc.enable` or unexpected behavior will occur
|
||||
enable = false
|
||||
|
||||
@@ -762,24 +762,6 @@ apply_on_query = "auto"
|
||||
## - `[size]` e.g. `64MB`: fixed memory threshold
|
||||
mem_threshold_on_create = "auto"
|
||||
|
||||
[region_engine.mito.memtable]
|
||||
## Memtable type.
|
||||
## - `time_series`: time-series memtable
|
||||
## - `partition_tree`: partition tree memtable (experimental)
|
||||
type = "time_series"
|
||||
|
||||
## The max number of keys in one shard.
|
||||
## Only available for `partition_tree` memtable.
|
||||
index_max_keys_per_shard = 8192
|
||||
|
||||
## The max rows of data inside the actively writing buffer in one shard.
|
||||
## Only available for `partition_tree` memtable.
|
||||
data_freeze_threshold = 32768
|
||||
|
||||
## Max dictionary bytes.
|
||||
## Only available for `partition_tree` memtable.
|
||||
fork_dictionary_bytes = "1GiB"
|
||||
|
||||
[[region_engine]]
|
||||
## Enable the file engine.
|
||||
[region_engine.file]
|
||||
|
||||
@@ -14,20 +14,14 @@
|
||||
|
||||
use std::collections::HashSet;
|
||||
|
||||
use api::v1::{ArrowIpc, ColumnDataType, SemanticType};
|
||||
use api::v1::{ArrowIpc, SemanticType};
|
||||
use bytes::Bytes;
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_error::status_code::StatusCode;
|
||||
use common_grpc::flight::{FlightEncoder, FlightMessage};
|
||||
use common_query::prelude::{greptime_timestamp, greptime_value};
|
||||
use datatypes::arrow::array::{Array, Float64Array, StringArray, TimestampMillisecondArray};
|
||||
use datatypes::arrow::record_batch::RecordBatch;
|
||||
use snafu::{OptionExt, ensure};
|
||||
use store_api::codec::PrimaryKeyEncoding;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::region_request::{
|
||||
AffectedRows, RegionBulkInsertsRequest, RegionPutRequest, RegionRequest,
|
||||
};
|
||||
use store_api::region_request::{AffectedRows, RegionBulkInsertsRequest, RegionRequest};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::batch_modifier::{TagColumnInfo, modify_batch_sparse};
|
||||
@@ -42,8 +36,7 @@ impl MetricEngineInner {
|
||||
/// **Logical region path:** The request payload is a logical `RecordBatch`
|
||||
/// (timestamp, value and tag columns). It is transformed to physical format
|
||||
/// via `modify_batch_sparse`, encoded to Arrow IPC, and forwarded as a
|
||||
/// `BulkInserts` request to the data region. If mito reports
|
||||
/// `StatusCode::Unsupported`, the request is transparently retried as a `Put`.
|
||||
/// `BulkInserts` request to the data region.
|
||||
///
|
||||
/// **Physical region path:** The request payload is already in physical format
|
||||
/// (produced by the batcher's `flush_batch_physical`). It is forwarded directly
|
||||
@@ -134,27 +127,9 @@ impl MetricEngineInner {
|
||||
},
|
||||
partition_expr_version,
|
||||
};
|
||||
match self
|
||||
.data_region
|
||||
self.data_region
|
||||
.write_data(data_region_id, RegionRequest::BulkInserts(request))
|
||||
.await
|
||||
{
|
||||
Ok(affected_rows) => Ok(affected_rows),
|
||||
Err(err) if err.status_code() == StatusCode::Unsupported => {
|
||||
// todo(hl): fallback path for PartitionTreeMemtable, remove this once we remove it
|
||||
let rows = record_batch_to_rows(&batch, region_id)?;
|
||||
self.put_region(
|
||||
region_id,
|
||||
RegionPutRequest {
|
||||
rows,
|
||||
hint: None,
|
||||
partition_expr_version,
|
||||
},
|
||||
)
|
||||
.await
|
||||
}
|
||||
Err(err) => Err(err),
|
||||
}
|
||||
}
|
||||
|
||||
fn resolve_tag_columns_from_metadata(
|
||||
@@ -214,174 +189,6 @@ impl MetricEngineInner {
|
||||
}
|
||||
}
|
||||
|
||||
fn record_batch_to_rows(batch: &RecordBatch, logical_region_id: RegionId) -> Result<api::v1::Rows> {
|
||||
let schema_ref = batch.schema();
|
||||
let fields = schema_ref.fields();
|
||||
|
||||
let mut ts_idx = None;
|
||||
let mut val_idx = None;
|
||||
let mut tag_indices = Vec::new();
|
||||
|
||||
for (idx, field) in fields.iter().enumerate() {
|
||||
if field.name() == greptime_timestamp() {
|
||||
ts_idx = Some(idx);
|
||||
if !matches!(
|
||||
field.data_type(),
|
||||
datatypes::arrow::datatypes::DataType::Timestamp(
|
||||
datatypes::arrow::datatypes::TimeUnit::Millisecond,
|
||||
_
|
||||
)
|
||||
) {
|
||||
return error::UnexpectedRequestSnafu {
|
||||
reason: format!(
|
||||
"Timestamp column '{}' in region {:?} has incompatible type: {:?}",
|
||||
field.name(),
|
||||
logical_region_id,
|
||||
field.data_type()
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
} else if field.name() == greptime_value() {
|
||||
val_idx = Some(idx);
|
||||
if !matches!(
|
||||
field.data_type(),
|
||||
datatypes::arrow::datatypes::DataType::Float64
|
||||
) {
|
||||
return error::UnexpectedRequestSnafu {
|
||||
reason: format!(
|
||||
"Value column '{}' in region {:?} has incompatible type: {:?}",
|
||||
field.name(),
|
||||
logical_region_id,
|
||||
field.data_type()
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
} else {
|
||||
if !matches!(
|
||||
field.data_type(),
|
||||
datatypes::arrow::datatypes::DataType::Utf8
|
||||
) {
|
||||
return error::UnexpectedRequestSnafu {
|
||||
reason: format!(
|
||||
"Tag column '{}' in region {:?} must be Utf8, found: {:?}",
|
||||
field.name(),
|
||||
logical_region_id,
|
||||
field.data_type()
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
tag_indices.push(idx);
|
||||
}
|
||||
}
|
||||
|
||||
let ts_idx = ts_idx.with_context(|| error::UnexpectedRequestSnafu {
|
||||
reason: format!(
|
||||
"Timestamp column '{}' not found in RecordBatch for region {:?}",
|
||||
greptime_timestamp(),
|
||||
logical_region_id
|
||||
),
|
||||
})?;
|
||||
let val_idx = val_idx.with_context(|| error::UnexpectedRequestSnafu {
|
||||
reason: format!(
|
||||
"Value column '{}' not found in RecordBatch for region {:?}",
|
||||
greptime_value(),
|
||||
logical_region_id
|
||||
),
|
||||
})?;
|
||||
|
||||
let mut schema = Vec::with_capacity(2 + tag_indices.len());
|
||||
schema.push(api::v1::ColumnSchema {
|
||||
column_name: greptime_timestamp().to_string(),
|
||||
datatype: ColumnDataType::TimestampMillisecond as i32,
|
||||
semantic_type: SemanticType::Timestamp as i32,
|
||||
datatype_extension: None,
|
||||
options: None,
|
||||
});
|
||||
schema.push(api::v1::ColumnSchema {
|
||||
column_name: greptime_value().to_string(),
|
||||
datatype: ColumnDataType::Float64 as i32,
|
||||
semantic_type: SemanticType::Field as i32,
|
||||
datatype_extension: None,
|
||||
options: None,
|
||||
});
|
||||
for &idx in &tag_indices {
|
||||
let field = &fields[idx];
|
||||
schema.push(api::v1::ColumnSchema {
|
||||
column_name: field.name().clone(),
|
||||
datatype: ColumnDataType::String as i32,
|
||||
semantic_type: SemanticType::Tag as i32,
|
||||
datatype_extension: None,
|
||||
options: None,
|
||||
});
|
||||
}
|
||||
|
||||
let ts_array = batch
|
||||
.column(ts_idx)
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampMillisecondArray>()
|
||||
.expect("validated as TimestampMillisecond");
|
||||
let val_array = batch
|
||||
.column(val_idx)
|
||||
.as_any()
|
||||
.downcast_ref::<Float64Array>()
|
||||
.expect("validated as Float64");
|
||||
let tag_arrays: Vec<&StringArray> = tag_indices
|
||||
.iter()
|
||||
.map(|&idx| {
|
||||
batch
|
||||
.column(idx)
|
||||
.as_any()
|
||||
.downcast_ref::<StringArray>()
|
||||
.expect("validated as Utf8")
|
||||
})
|
||||
.collect();
|
||||
|
||||
let num_rows = batch.num_rows();
|
||||
let mut rows = Vec::with_capacity(num_rows);
|
||||
for row_idx in 0..num_rows {
|
||||
let mut values = Vec::with_capacity(2 + tag_arrays.len());
|
||||
|
||||
if ts_array.is_null(row_idx) {
|
||||
values.push(api::v1::Value { value_data: None });
|
||||
} else {
|
||||
values.push(api::v1::Value {
|
||||
value_data: Some(api::v1::value::ValueData::TimestampMillisecondValue(
|
||||
ts_array.value(row_idx),
|
||||
)),
|
||||
});
|
||||
}
|
||||
|
||||
if val_array.is_null(row_idx) {
|
||||
values.push(api::v1::Value { value_data: None });
|
||||
} else {
|
||||
values.push(api::v1::Value {
|
||||
value_data: Some(api::v1::value::ValueData::F64Value(
|
||||
val_array.value(row_idx),
|
||||
)),
|
||||
});
|
||||
}
|
||||
|
||||
for arr in &tag_arrays {
|
||||
if arr.is_null(row_idx) {
|
||||
values.push(api::v1::Value { value_data: None });
|
||||
} else {
|
||||
values.push(api::v1::Value {
|
||||
value_data: Some(api::v1::value::ValueData::StringValue(
|
||||
arr.value(row_idx).to_string(),
|
||||
)),
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
rows.push(api::v1::Row { values });
|
||||
}
|
||||
|
||||
Ok(api::v1::Rows { schema, rows })
|
||||
}
|
||||
|
||||
fn record_batch_to_ipc(record_batch: &RecordBatch) -> Result<(Bytes, Bytes, Bytes)> {
|
||||
let mut encoder = FlightEncoder::default();
|
||||
let schema = encoder.encode_schema(record_batch.schema().as_ref());
|
||||
@@ -422,7 +229,7 @@ mod tests {
|
||||
use datatypes::arrow::datatypes::{DataType, Field, Schema as ArrowSchema, TimeUnit};
|
||||
use datatypes::arrow::record_batch::RecordBatch;
|
||||
use mito2::config::MitoConfig;
|
||||
use store_api::metric_engine_consts::MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING;
|
||||
use store_api::metric_engine_consts::PRIMARY_KEY_ENCODING;
|
||||
use store_api::path_utils::table_dir;
|
||||
use store_api::region_engine::RegionEngine;
|
||||
use store_api::region_request::{RegionBulkInsertsRequest, RegionPutRequest, RegionRequest};
|
||||
@@ -483,10 +290,7 @@ mod tests {
|
||||
env.create_physical_region(
|
||||
physical_region_id,
|
||||
&TestEnv::default_table_dir(),
|
||||
vec![(
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING.to_string(),
|
||||
"dense".to_string(),
|
||||
)],
|
||||
vec![(PRIMARY_KEY_ENCODING.to_string(), "dense".to_string())],
|
||||
)
|
||||
.await;
|
||||
|
||||
@@ -810,65 +614,4 @@ mod tests {
|
||||
|
||||
assert_eq!(put_output, bulk_output);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_record_batch_to_rows_with_null_values() {
|
||||
use datatypes::arrow::array::{Float64Array, StringArray, TimestampMillisecondArray};
|
||||
use datatypes::arrow::datatypes::{DataType, Field, Schema as ArrowSchema, TimeUnit};
|
||||
use datatypes::arrow::record_batch::RecordBatch;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::engine::bulk_insert::record_batch_to_rows;
|
||||
|
||||
let schema = Arc::new(ArrowSchema::new(vec![
|
||||
Field::new(
|
||||
greptime_timestamp(),
|
||||
DataType::Timestamp(TimeUnit::Millisecond, None),
|
||||
true,
|
||||
),
|
||||
Field::new(greptime_value(), DataType::Float64, true),
|
||||
Field::new("job", DataType::Utf8, true),
|
||||
Field::new("host", DataType::Utf8, true),
|
||||
]));
|
||||
|
||||
let ts_array = TimestampMillisecondArray::from(vec![Some(1000), None, Some(3000)]);
|
||||
let val_array = Float64Array::from(vec![Some(1.0), Some(2.0), None]);
|
||||
let job_array = StringArray::from(vec![Some("job1"), None, Some("job3")]);
|
||||
let host_array = StringArray::from(vec![None, Some("host2"), Some("host3")]);
|
||||
|
||||
let batch = RecordBatch::try_new(
|
||||
schema,
|
||||
vec![
|
||||
Arc::new(ts_array),
|
||||
Arc::new(val_array),
|
||||
Arc::new(job_array),
|
||||
Arc::new(host_array),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let rows = record_batch_to_rows(&batch, region_id).unwrap();
|
||||
|
||||
assert_eq!(rows.rows.len(), 3);
|
||||
assert_eq!(rows.schema.len(), 4);
|
||||
|
||||
// Row 0: all non-null except host
|
||||
assert!(rows.rows[0].values[0].value_data.is_some());
|
||||
assert!(rows.rows[0].values[1].value_data.is_some());
|
||||
assert!(rows.rows[0].values[2].value_data.is_some());
|
||||
assert!(rows.rows[0].values[3].value_data.is_none());
|
||||
|
||||
// Row 1: null timestamp, null job
|
||||
assert!(rows.rows[1].values[0].value_data.is_none());
|
||||
assert!(rows.rows[1].values[1].value_data.is_some());
|
||||
assert!(rows.rows[1].values[2].value_data.is_none());
|
||||
assert!(rows.rows[1].values[3].value_data.is_some());
|
||||
|
||||
// Row 2: null value
|
||||
assert!(rows.rows[2].values[0].value_data.is_some());
|
||||
assert!(rows.rows[2].values[1].value_data.is_none());
|
||||
assert!(rows.rows[2].values[2].value_data.is_some());
|
||||
assert!(rows.rows[2].values[3].value_data.is_some());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,9 +22,14 @@ use store_api::metric_engine_consts::{
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION_DEFAULT,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION_DEFAULT, METRIC_ENGINE_INDEX_TYPE_OPTION,
|
||||
PRIMARY_KEY_ENCODING,
|
||||
};
|
||||
use store_api::mito_engine_options::{COMPACTION_TYPE, COMPACTION_TYPE_TWCS, TWCS_TIME_WINDOW};
|
||||
|
||||
/// Prefix for legacy `memtable.partition_tree.*` option keys. These keys are
|
||||
/// silently dropped by the metric engine; the partition tree memtable is gone.
|
||||
const LEGACY_PARTITION_TREE_OPTION_PREFIX: &str = "memtable.partition_tree.";
|
||||
|
||||
use crate::error::{Error, ParseRegionOptionsSnafu, Result};
|
||||
|
||||
/// The empirical value for the seg row count of the metric data region.
|
||||
@@ -66,16 +71,27 @@ pub fn set_data_region_options(
|
||||
"index.inverted_index.segment_row_count".to_string(),
|
||||
SEG_ROW_COUNT_FOR_DATA_REGION.to_string(),
|
||||
);
|
||||
// Set memtable options for the data region.
|
||||
options.insert("memtable.type".to_string(), "partition_tree".to_string());
|
||||
if sparse_primary_key_encoding_if_absent
|
||||
&& !options.contains_key(MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING)
|
||||
{
|
||||
options.insert(
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING.to_string(),
|
||||
"sparse".to_string(),
|
||||
);
|
||||
|
||||
// Extract primary key encoding from the legacy nested key before dropping
|
||||
// all `memtable.partition_tree.*` keys.
|
||||
let legacy_encoding = options.remove(MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING);
|
||||
options.retain(|k, _| !k.starts_with(LEGACY_PARTITION_TREE_OPTION_PREFIX));
|
||||
|
||||
// Set memtable options for the data region. Bulk memtable produces
|
||||
// flat-encoded ranges, so the SST format must be flat to match.
|
||||
options.insert("memtable.type".to_string(), "bulk".to_string());
|
||||
options.insert("sst_format".to_string(), "flat".to_string());
|
||||
|
||||
// Decide the top-level primary key encoding: caller-supplied top-level key wins,
|
||||
// then extracted legacy value, then the `sparse` default if requested.
|
||||
if !options.contains_key(PRIMARY_KEY_ENCODING) {
|
||||
if let Some(encoding) = legacy_encoding {
|
||||
options.insert(PRIMARY_KEY_ENCODING.to_string(), encoding);
|
||||
} else if sparse_primary_key_encoding_if_absent {
|
||||
options.insert(PRIMARY_KEY_ENCODING.to_string(), "sparse".to_string());
|
||||
}
|
||||
}
|
||||
|
||||
if !options.contains_key(TWCS_TIME_WINDOW) {
|
||||
options.insert(
|
||||
COMPACTION_TYPE.to_string(),
|
||||
@@ -213,6 +229,8 @@ mod tests {
|
||||
let mut options = HashMap::new();
|
||||
set_data_region_options(&mut options, false);
|
||||
|
||||
assert_eq!(options.get("memtable.type"), Some(&"bulk".to_string()));
|
||||
assert_eq!(options.get("sst_format"), Some(&"flat".to_string()));
|
||||
assert_eq!(
|
||||
options.get(COMPACTION_TYPE),
|
||||
Some(&COMPACTION_TYPE_TWCS.to_string())
|
||||
@@ -220,6 +238,58 @@ mod tests {
|
||||
assert_eq!(options.get(TWCS_TIME_WINDOW), Some(&"1d".to_string()));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_set_data_region_options_sparse_primary_key_encoding() {
|
||||
let mut options = HashMap::new();
|
||||
set_data_region_options(&mut options, true);
|
||||
|
||||
assert_eq!(options.get("memtable.type"), Some(&"bulk".to_string()));
|
||||
assert_eq!(options.get("sst_format"), Some(&"flat".to_string()));
|
||||
assert_eq!(
|
||||
options.get(PRIMARY_KEY_ENCODING),
|
||||
Some(&"sparse".to_string())
|
||||
);
|
||||
assert!(!options.contains_key(MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_set_data_region_options_migrates_legacy_partition_tree_options() {
|
||||
let mut options = HashMap::new();
|
||||
options.insert("memtable.type".to_string(), "partition_tree".to_string());
|
||||
options.insert(
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING.to_string(),
|
||||
"sparse".to_string(),
|
||||
);
|
||||
options.insert(
|
||||
"memtable.partition_tree.index_max_keys_per_shard".to_string(),
|
||||
"2048".to_string(),
|
||||
);
|
||||
set_data_region_options(&mut options, false);
|
||||
|
||||
assert_eq!(options.get("memtable.type"), Some(&"bulk".to_string()));
|
||||
assert_eq!(options.get("sst_format"), Some(&"flat".to_string()));
|
||||
assert_eq!(
|
||||
options.get(PRIMARY_KEY_ENCODING),
|
||||
Some(&"sparse".to_string())
|
||||
);
|
||||
// All legacy partition-tree-specific keys should be stripped.
|
||||
assert!(!options.contains_key(MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING));
|
||||
assert!(!options.contains_key("memtable.partition_tree.index_max_keys_per_shard"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_set_data_region_options_preserves_existing_top_level_encoding() {
|
||||
let mut options = HashMap::new();
|
||||
options.insert(PRIMARY_KEY_ENCODING.to_string(), "dense".to_string());
|
||||
// Sparse flag is on but caller already specified dense.
|
||||
set_data_region_options(&mut options, true);
|
||||
|
||||
assert_eq!(
|
||||
options.get(PRIMARY_KEY_ENCODING),
|
||||
Some(&"dense".to_string())
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_set_data_region_options_respects_user_compaction_time_window() {
|
||||
// Test that user-specified time window is preserved
|
||||
|
||||
@@ -709,8 +709,7 @@ mod tests {
|
||||
use partition::expr::col;
|
||||
use store_api::metadata::ColumnMetadata;
|
||||
use store_api::metric_engine_consts::{
|
||||
DATA_SCHEMA_TABLE_ID_COLUMN_NAME, DATA_SCHEMA_TSID_COLUMN_NAME,
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING,
|
||||
DATA_SCHEMA_TABLE_ID_COLUMN_NAME, DATA_SCHEMA_TSID_COLUMN_NAME, PRIMARY_KEY_ENCODING,
|
||||
};
|
||||
use store_api::path_utils::table_dir;
|
||||
use store_api::region_engine::RegionEngine;
|
||||
@@ -1282,10 +1281,7 @@ mod tests {
|
||||
run_batch_write_with_schema_variants(
|
||||
&env,
|
||||
physical_region_id,
|
||||
vec![(
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING.to_string(),
|
||||
"sparse".to_string(),
|
||||
)],
|
||||
vec![(PRIMARY_KEY_ENCODING.to_string(), "sparse".to_string())],
|
||||
true,
|
||||
)
|
||||
.await;
|
||||
@@ -1299,10 +1295,7 @@ mod tests {
|
||||
run_batch_write_with_schema_variants(
|
||||
&env,
|
||||
physical_region_id,
|
||||
vec![(
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING.to_string(),
|
||||
"dense".to_string(),
|
||||
)],
|
||||
vec![(PRIMARY_KEY_ENCODING.to_string(), "dense".to_string())],
|
||||
false,
|
||||
)
|
||||
.await;
|
||||
|
||||
@@ -247,12 +247,12 @@ fn bench_primary_key_filter(c: &mut Criterion) {
|
||||
|
||||
let dense_pk = encode_dense_pk(&metadata, &row);
|
||||
let dense_codec = DensePrimaryKeyCodec::new(&metadata);
|
||||
let mut dense_fast = dense_codec.primary_key_filter(&metadata, filters.clone(), false);
|
||||
let mut dense_fast = dense_codec.primary_key_filter(&metadata, filters.clone());
|
||||
let mut dense_offsets = Vec::new();
|
||||
|
||||
let sparse_pk = encode_sparse_pk(&metadata, &row);
|
||||
let sparse_codec = SparsePrimaryKeyCodec::new(&metadata);
|
||||
let mut sparse_fast = sparse_codec.primary_key_filter(&metadata, filters.clone(), false);
|
||||
let mut sparse_fast = sparse_codec.primary_key_filter(&metadata, filters.clone());
|
||||
let mut sparse_offsets = SparseOffsetsCache::new();
|
||||
|
||||
let mut group = c.benchmark_group(format!("primary_key_filter/{case_name}"));
|
||||
|
||||
@@ -21,7 +21,6 @@ use datatypes::value::Value;
|
||||
use memcomparable::Serializer;
|
||||
use snafu::ResultExt;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME;
|
||||
use store_api::storage::ColumnId;
|
||||
|
||||
use crate::error::{EvaluateFilterSnafu, Result};
|
||||
@@ -29,11 +28,6 @@ use crate::row_converter::{
|
||||
DensePrimaryKeyCodec, PrimaryKeyFilter, SortField, SparseOffsetsCache, SparsePrimaryKeyCodec,
|
||||
};
|
||||
|
||||
/// Returns true if this is a partition column for metrics in the memtable.
|
||||
pub fn is_partition_column(name: &str) -> bool {
|
||||
name == DATA_SCHEMA_TABLE_ID_COLUMN_NAME
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
struct PrimaryKeyFilterInner {
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
@@ -41,12 +35,8 @@ struct PrimaryKeyFilterInner {
|
||||
}
|
||||
|
||||
impl PrimaryKeyFilterInner {
|
||||
fn new(
|
||||
metadata: RegionMetadataRef,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
skip_partition_column: bool,
|
||||
) -> Self {
|
||||
let compiled_filters = Self::compile_filters(&metadata, &filters, skip_partition_column);
|
||||
fn new(metadata: RegionMetadataRef, filters: Arc<Vec<SimpleFilterEvaluator>>) -> Self {
|
||||
let compiled_filters = Self::compile_filters(&metadata, &filters);
|
||||
Self {
|
||||
filters,
|
||||
compiled_filters,
|
||||
@@ -56,7 +46,6 @@ impl PrimaryKeyFilterInner {
|
||||
fn compile_filters(
|
||||
metadata: &RegionMetadataRef,
|
||||
filters: &[SimpleFilterEvaluator],
|
||||
skip_partition_column: bool,
|
||||
) -> Vec<CompiledPrimaryKeyFilter> {
|
||||
if filters.is_empty() || metadata.primary_key.is_empty() {
|
||||
return Vec::new();
|
||||
@@ -64,10 +53,6 @@ impl PrimaryKeyFilterInner {
|
||||
|
||||
let mut compiled_filters = Vec::with_capacity(filters.len());
|
||||
for (filter_idx, filter) in filters.iter().enumerate() {
|
||||
if skip_partition_column && is_partition_column(filter.column_name()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
let Some(column) = metadata.column_by_name(filter.column_name()) else {
|
||||
continue;
|
||||
};
|
||||
@@ -256,10 +241,9 @@ impl DensePrimaryKeyFilter {
|
||||
metadata: RegionMetadataRef,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
codec: DensePrimaryKeyCodec,
|
||||
skip_partition_column: bool,
|
||||
) -> Self {
|
||||
Self {
|
||||
inner: PrimaryKeyFilterInner::new(metadata, filters, skip_partition_column),
|
||||
inner: PrimaryKeyFilterInner::new(metadata, filters),
|
||||
codec,
|
||||
offsets_buf: Vec::new(),
|
||||
}
|
||||
@@ -310,10 +294,9 @@ impl SparsePrimaryKeyFilter {
|
||||
metadata: RegionMetadataRef,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
codec: SparsePrimaryKeyCodec,
|
||||
skip_partition_column: bool,
|
||||
) -> Self {
|
||||
Self {
|
||||
inner: PrimaryKeyFilterInner::new(metadata, filters, skip_partition_column),
|
||||
inner: PrimaryKeyFilterInner::new(metadata, filters),
|
||||
codec,
|
||||
offsets_cache: SparseOffsetsCache::new(),
|
||||
}
|
||||
@@ -513,7 +496,7 @@ mod tests {
|
||||
)]);
|
||||
let pk = encode_sparse_pk(&metadata, 1, 0, create_test_row());
|
||||
let codec = SparsePrimaryKeyCodec::new(&metadata);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -526,7 +509,7 @@ mod tests {
|
||||
)]);
|
||||
let pk = encode_sparse_pk(&metadata, 1, 0, create_test_row());
|
||||
let codec = SparsePrimaryKeyCodec::new(&metadata);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
assert!(!filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -539,7 +522,7 @@ mod tests {
|
||||
)]);
|
||||
let pk = encode_sparse_pk(&metadata, 1, 0, create_test_row());
|
||||
let codec = SparsePrimaryKeyCodec::new(&metadata);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -552,7 +535,7 @@ mod tests {
|
||||
)]);
|
||||
let pk = encode_dense_pk(&metadata, create_test_row());
|
||||
let codec = DensePrimaryKeyCodec::new(&metadata);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -565,7 +548,7 @@ mod tests {
|
||||
)]);
|
||||
let pk = encode_dense_pk(&metadata, create_test_row());
|
||||
let codec = DensePrimaryKeyCodec::new(&metadata);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
assert!(!filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -578,7 +561,7 @@ mod tests {
|
||||
)]);
|
||||
let pk = encode_dense_pk(&metadata, create_test_row());
|
||||
let codec = DensePrimaryKeyCodec::new(&metadata);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -597,8 +580,7 @@ mod tests {
|
||||
|
||||
for (op, value, expected) in cases {
|
||||
let filters = Arc::new(vec![create_filter_with_op("pod", op, value)]);
|
||||
let mut filter =
|
||||
DensePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone(), false);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone());
|
||||
assert_eq!(expected, filter.matches(&pk).unwrap());
|
||||
}
|
||||
}
|
||||
@@ -618,8 +600,7 @@ mod tests {
|
||||
|
||||
for (op, value, expected) in cases {
|
||||
let filters = Arc::new(vec![create_filter_with_op("pod", op, value)]);
|
||||
let mut filter =
|
||||
SparsePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone(), false);
|
||||
let mut filter = SparsePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone());
|
||||
assert_eq!(expected, filter.matches(&pk).unwrap());
|
||||
}
|
||||
}
|
||||
@@ -652,7 +633,7 @@ mod tests {
|
||||
.unwrap();
|
||||
|
||||
let filters = Arc::new(vec![create_filter_with_op("f", Operator::Eq, 0.0_f64)]);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
@@ -674,29 +655,7 @@ mod tests {
|
||||
Operator::Eq,
|
||||
42_u32,
|
||||
)]);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false);
|
||||
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_dense_primary_key_filter_can_skip_partition_column() {
|
||||
let metadata = setup_partitioned_metadata();
|
||||
let codec = DensePrimaryKeyCodec::new(&metadata);
|
||||
let mut pk = Vec::new();
|
||||
codec
|
||||
.encode_to_vec(
|
||||
[ValueRef::UInt32(42), ValueRef::String("host-a")].into_iter(),
|
||||
&mut pk,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let filters = Arc::new(vec![create_filter_with_op(
|
||||
DATA_SCHEMA_TABLE_ID_COLUMN_NAME,
|
||||
Operator::Eq,
|
||||
7_u32,
|
||||
)]);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, true);
|
||||
let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec);
|
||||
|
||||
assert!(filter.matches(&pk).unwrap());
|
||||
}
|
||||
|
||||
@@ -124,7 +124,6 @@ pub trait PrimaryKeyCodec: Send + Sync + Debug {
|
||||
&self,
|
||||
metadata: &RegionMetadataRef,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
skip_partition_column: bool,
|
||||
) -> Box<dyn PrimaryKeyFilter>;
|
||||
|
||||
/// Returns the estimated size of the primary key.
|
||||
|
||||
@@ -556,13 +556,11 @@ impl PrimaryKeyCodec for DensePrimaryKeyCodec {
|
||||
&self,
|
||||
metadata: &RegionMetadataRef,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
skip_partition_column: bool,
|
||||
) -> Box<dyn PrimaryKeyFilter> {
|
||||
Box::new(DensePrimaryKeyFilter::new(
|
||||
metadata.clone(),
|
||||
filters,
|
||||
self.clone(),
|
||||
skip_partition_column,
|
||||
))
|
||||
}
|
||||
|
||||
|
||||
@@ -507,13 +507,11 @@ impl PrimaryKeyCodec for SparsePrimaryKeyCodec {
|
||||
&self,
|
||||
metadata: &RegionMetadataRef,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
skip_partition_column: bool,
|
||||
) -> Box<dyn PrimaryKeyFilter> {
|
||||
Box::new(SparsePrimaryKeyFilter::new(
|
||||
metadata.clone(),
|
||||
filters,
|
||||
self.clone(),
|
||||
skip_partition_column,
|
||||
))
|
||||
}
|
||||
|
||||
|
||||
@@ -28,7 +28,6 @@ use mito2::memtable::bulk::context::BulkIterContext;
|
||||
use mito2::memtable::bulk::part::BulkPartConverter;
|
||||
use mito2::memtable::bulk::part_reader::BulkPartBatchIter;
|
||||
use mito2::memtable::bulk::{BulkMemtable, BulkMemtableConfig};
|
||||
use mito2::memtable::partition_tree::{PartitionTreeConfig, PartitionTreeMemtable};
|
||||
use mito2::memtable::time_series::TimeSeriesMemtable;
|
||||
use mito2::memtable::{IterBuilder, Memtable, RangesOptions};
|
||||
use mito2::read::flat_merge::FlatMergeIterator;
|
||||
@@ -45,21 +44,6 @@ fn write_rows(c: &mut Criterion) {
|
||||
|
||||
// Note that this test only generate one time series.
|
||||
let mut group = c.benchmark_group("write");
|
||||
group.bench_function("partition_tree", |b| {
|
||||
let codec = Arc::new(DensePrimaryKeyCodec::new(&metadata));
|
||||
let memtable = PartitionTreeMemtable::new(
|
||||
1,
|
||||
codec,
|
||||
metadata.clone(),
|
||||
None,
|
||||
&PartitionTreeConfig::default(),
|
||||
);
|
||||
let kvs =
|
||||
memtable_util::build_key_values(&metadata, "hello".to_string(), 42, ×tamps, 1);
|
||||
b.iter(|| {
|
||||
memtable.write(&kvs).unwrap();
|
||||
});
|
||||
});
|
||||
group.bench_function("time_series", |b| {
|
||||
let memtable = TimeSeriesMemtable::new(metadata.clone(), 1, None, true, MergeMode::LastRow);
|
||||
let kvs =
|
||||
@@ -73,26 +57,11 @@ fn write_rows(c: &mut Criterion) {
|
||||
/// Scans all rows.
|
||||
fn full_scan(c: &mut Criterion) {
|
||||
let metadata = Arc::new(cpu_metadata());
|
||||
let config = PartitionTreeConfig::default();
|
||||
let start_sec = 1710043200;
|
||||
let generator = CpuDataGenerator::new(metadata.clone(), 4000, start_sec, start_sec + 3600 * 2);
|
||||
|
||||
let mut group = c.benchmark_group("full_scan");
|
||||
group.sample_size(10);
|
||||
group.bench_function("partition_tree", |b| {
|
||||
let codec = Arc::new(DensePrimaryKeyCodec::new(&metadata));
|
||||
let memtable = PartitionTreeMemtable::new(1, codec, metadata.clone(), None, &config);
|
||||
for kvs in generator.iter() {
|
||||
memtable.write(&kvs).unwrap();
|
||||
}
|
||||
|
||||
b.iter(|| {
|
||||
let iter = memtable.iter(None, None, None).unwrap();
|
||||
for batch in iter {
|
||||
let _batch = batch.unwrap();
|
||||
}
|
||||
});
|
||||
});
|
||||
group.bench_function("time_series", |b| {
|
||||
let memtable = TimeSeriesMemtable::new(metadata.clone(), 1, None, true, MergeMode::LastRow);
|
||||
for kvs in generator.iter() {
|
||||
@@ -115,27 +84,11 @@ fn full_scan(c: &mut Criterion) {
|
||||
/// Filters 1 host.
|
||||
fn filter_1_host(c: &mut Criterion) {
|
||||
let metadata = Arc::new(cpu_metadata());
|
||||
let config = PartitionTreeConfig::default();
|
||||
let start_sec = 1710043200;
|
||||
let generator = CpuDataGenerator::new(metadata.clone(), 4000, start_sec, start_sec + 3600 * 2);
|
||||
|
||||
let mut group = c.benchmark_group("filter_1_host");
|
||||
group.sample_size(10);
|
||||
group.bench_function("partition_tree", |b| {
|
||||
let codec = Arc::new(DensePrimaryKeyCodec::new(&metadata));
|
||||
let memtable = PartitionTreeMemtable::new(1, codec, metadata.clone(), None, &config);
|
||||
for kvs in generator.iter() {
|
||||
memtable.write(&kvs).unwrap();
|
||||
}
|
||||
let predicate = generator.random_host_filter();
|
||||
|
||||
b.iter(|| {
|
||||
let iter = memtable.iter(None, Some(predicate.clone()), None).unwrap();
|
||||
for batch in iter {
|
||||
let _batch = batch.unwrap();
|
||||
}
|
||||
});
|
||||
});
|
||||
group.bench_function("time_series", |b| {
|
||||
let memtable = TimeSeriesMemtable::new(metadata.clone(), 1, None, true, MergeMode::LastRow);
|
||||
for kvs in generator.iter() {
|
||||
|
||||
@@ -42,7 +42,7 @@ use datafusion_expr::Expr;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::storage::{RegionId, TableId};
|
||||
use store_api::storage::RegionId;
|
||||
use task::MAX_PARALLEL_COMPACTION;
|
||||
use tokio::sync::mpsc::{self, Sender};
|
||||
|
||||
@@ -452,7 +452,7 @@ impl CompactionScheduler {
|
||||
) -> Result<Option<ActiveCompaction>> {
|
||||
let region_id = request.region_id();
|
||||
let (dynamic_compaction_opts, ttl) = find_dynamic_options(
|
||||
region_id.table_id(),
|
||||
region_id,
|
||||
&request.current_version.options,
|
||||
&request.schema_metadata_manager,
|
||||
)
|
||||
@@ -715,10 +715,11 @@ impl Drop for CompactionScheduler {
|
||||
|
||||
/// Finds compaction options and TTL together with a single metadata fetch to reduce RTT.
|
||||
async fn find_dynamic_options(
|
||||
table_id: TableId,
|
||||
region_id: RegionId,
|
||||
region_options: &crate::region::options::RegionOptions,
|
||||
schema_metadata_manager: &SchemaMetadataManagerRef,
|
||||
) -> Result<(crate::region::options::CompactionOptions, TimeToLive)> {
|
||||
let table_id = region_id.table_id();
|
||||
if let (true, Some(ttl)) = (region_options.compaction_override, region_options.ttl) {
|
||||
debug!(
|
||||
"Use region options directly for table {}: compaction={:?}, ttl={:?}",
|
||||
@@ -765,7 +766,7 @@ async fn find_dynamic_options(
|
||||
if map.is_empty() {
|
||||
region_options.compaction.clone()
|
||||
} else {
|
||||
crate::region::options::RegionOptions::try_from(&map)
|
||||
crate::region::options::RegionOptions::try_from_options(region_id, &map)
|
||||
.map(|o| o.compaction)
|
||||
.unwrap_or_else(|e| {
|
||||
error!(e; "Failed to create RegionOptions from map");
|
||||
@@ -1199,7 +1200,7 @@ mod tests {
|
||||
|
||||
let version_control = Arc::new(builder.build());
|
||||
let region_opts = version_control.current().version.options.clone();
|
||||
let (opts, _) = find_dynamic_options(table_id, ®ion_opts, &schema_metadata_manager)
|
||||
let (opts, _) = find_dynamic_options(region_id, ®ion_opts, &schema_metadata_manager)
|
||||
.await
|
||||
.unwrap();
|
||||
match opts {
|
||||
@@ -1293,7 +1294,8 @@ mod tests {
|
||||
for (case_name, schema_value, override_set, table_window, expected_window) in cases {
|
||||
let builder = VersionControlBuilder::new();
|
||||
let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
|
||||
let table_id = builder.region_id().table_id();
|
||||
let region_id = builder.region_id();
|
||||
let table_id = region_id.table_id();
|
||||
schema_metadata_manager
|
||||
.register_region_table_info(
|
||||
table_id,
|
||||
@@ -1314,7 +1316,7 @@ mod tests {
|
||||
twcs.time_window = Some(window);
|
||||
}
|
||||
|
||||
let (opts, _) = find_dynamic_options(table_id, ®ion_opts, &schema_metadata_manager)
|
||||
let (opts, _) = find_dynamic_options(region_id, ®ion_opts, &schema_metadata_manager)
|
||||
.await
|
||||
.unwrap();
|
||||
match opts {
|
||||
|
||||
@@ -204,19 +204,16 @@ pub async fn open_compaction_region(
|
||||
Either::Left(ttl) => ttl,
|
||||
// Get the ttl from the schema metadata manager.
|
||||
Either::Right(schema_metadata_manager) => {
|
||||
let (_, ttl) = find_dynamic_options(
|
||||
req.region_id.table_id(),
|
||||
&req.region_options,
|
||||
&schema_metadata_manager,
|
||||
)
|
||||
.await
|
||||
.unwrap_or_else(|e| {
|
||||
warn!(e; "Failed to get ttl for region: {}", region_metadata.region_id);
|
||||
(
|
||||
crate::region::options::CompactionOptions::default(),
|
||||
TimeToLive::default(),
|
||||
)
|
||||
});
|
||||
let (_, ttl) =
|
||||
find_dynamic_options(req.region_id, &req.region_options, &schema_metadata_manager)
|
||||
.await
|
||||
.unwrap_or_else(|e| {
|
||||
warn!(e; "Failed to get ttl for region: {}", region_metadata.region_id);
|
||||
(
|
||||
crate::region::options::CompactionOptions::default(),
|
||||
TimeToLive::default(),
|
||||
)
|
||||
});
|
||||
ttl
|
||||
}
|
||||
};
|
||||
|
||||
@@ -252,6 +252,7 @@ mod tests {
|
||||
memtable: None,
|
||||
merge_mode: None,
|
||||
sst_format: None,
|
||||
primary_key_encoding: None,
|
||||
},
|
||||
compaction_time_window: None,
|
||||
}
|
||||
|
||||
@@ -29,7 +29,6 @@ use serde_with::serde_as;
|
||||
use crate::cache::file_cache::DEFAULT_INDEX_CACHE_PERCENT;
|
||||
use crate::error::Result;
|
||||
use crate::gc::GcConfig;
|
||||
use crate::memtable::MemtableConfig;
|
||||
use crate::sst::DEFAULT_WRITE_BUFFER_SIZE;
|
||||
|
||||
const MULTIPART_UPLOAD_MINIMUM_SIZE: ReadableSize = ReadableSize::mb(5);
|
||||
@@ -167,9 +166,6 @@ pub struct MitoConfig {
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub vector_index: VectorIndexConfig,
|
||||
|
||||
/// Memtable config
|
||||
pub memtable: MemtableConfig,
|
||||
|
||||
/// Minimum time interval between two compactions.
|
||||
/// To align with the old behavior, the default value is 0 (no restrictions).
|
||||
#[serde(with = "humantime_serde")]
|
||||
@@ -225,7 +221,6 @@ impl Default for MitoConfig {
|
||||
bloom_filter_index: BloomFilterConfig::default(),
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index: VectorIndexConfig::default(),
|
||||
memtable: MemtableConfig::default(),
|
||||
min_compaction_interval: Duration::from_secs(0),
|
||||
default_flat_format: true,
|
||||
gc: GcConfig::default(),
|
||||
@@ -700,25 +695,3 @@ fn divide_num_cpus(divisor: usize) -> usize {
|
||||
|
||||
cores.div_ceil(divisor)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_deserialize_config() {
|
||||
let s = r#"
|
||||
[memtable]
|
||||
type = "partition_tree"
|
||||
index_max_keys_per_shard = 8192
|
||||
data_freeze_threshold = 1024
|
||||
dedup = true
|
||||
fork_dictionary_bytes = "512MiB"
|
||||
"#;
|
||||
let config: MitoConfig = toml::from_str(s).unwrap();
|
||||
let MemtableConfig::PartitionTree(config) = &config.memtable else {
|
||||
unreachable!()
|
||||
};
|
||||
assert_eq!(1024, config.data_freeze_threshold);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -308,8 +308,8 @@ async fn test_engine_create_with_memtable_opts_with_format(flat_format: bool) {
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let request = CreateRequestBuilder::new()
|
||||
.insert_option("memtable.type", "partition_tree")
|
||||
.insert_option("memtable.partition_tree.index_max_keys_per_shard", "2")
|
||||
.insert_option("memtable.type", "bulk")
|
||||
.insert_option("memtable.bulk.merge_threshold", "7")
|
||||
.build();
|
||||
let column_schemas = rows_schema(&request);
|
||||
engine
|
||||
@@ -317,11 +317,16 @@ async fn test_engine_create_with_memtable_opts_with_format(flat_format: bool) {
|
||||
.await
|
||||
.unwrap();
|
||||
let region = engine.get_region(region_id).unwrap();
|
||||
let Some(MemtableOptions::PartitionTree(memtable_opts)) = ®ion.version().options.memtable
|
||||
else {
|
||||
unreachable!();
|
||||
};
|
||||
assert_eq!(2, memtable_opts.index_max_keys_per_shard);
|
||||
let version = region.version();
|
||||
match version
|
||||
.options
|
||||
.memtable
|
||||
.as_ref()
|
||||
.expect("memtable options should be set")
|
||||
{
|
||||
MemtableOptions::Bulk(config) => assert_eq!(config.merge_threshold, 7),
|
||||
other => panic!("expected bulk memtable, got {other:?}"),
|
||||
}
|
||||
|
||||
let rows = Rows {
|
||||
schema: column_schemas,
|
||||
|
||||
@@ -58,13 +58,6 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to encode sparse primary key, reason: {}", reason))]
|
||||
EncodeSparsePrimaryKey {
|
||||
reason: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("OpenDAL operator failed"))]
|
||||
OpenDal {
|
||||
#[snafu(implicit)]
|
||||
@@ -1346,7 +1339,6 @@ impl ErrorExt for Error {
|
||||
|
||||
WriteParquet { .. } => StatusCode::StorageUnavailable,
|
||||
WriteGroup { source, .. } => source.status_code(),
|
||||
EncodeSparsePrimaryKey { .. } => StatusCode::Unexpected,
|
||||
InvalidBatch { .. } => StatusCode::InvalidArguments,
|
||||
InvalidRecordBatch { .. } => StatusCode::InvalidArguments,
|
||||
ConvertVector { source, .. } => source.status_code(),
|
||||
|
||||
@@ -27,8 +27,8 @@ use datatypes::arrow::record_batch::RecordBatch;
|
||||
use mito_codec::key_values::KeyValue;
|
||||
pub use mito_codec::key_values::KeyValues;
|
||||
use mito_codec::row_converter::{PrimaryKeyCodec, build_primary_key_codec};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::ensure;
|
||||
use store_api::codec::PrimaryKeyEncoding;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::storage::{ColumnId, SequenceNumber, SequenceRange};
|
||||
|
||||
@@ -36,7 +36,6 @@ use crate::config::MitoConfig;
|
||||
use crate::error::{Result, UnsupportedOperationSnafu};
|
||||
use crate::flush::WriteBufferManagerRef;
|
||||
use crate::memtable::bulk::{BulkMemtableBuilder, CompactDispatcher};
|
||||
use crate::memtable::partition_tree::{PartitionTreeConfig, PartitionTreeMemtableBuilder};
|
||||
use crate::memtable::time_series::TimeSeriesMemtableBuilder;
|
||||
use crate::metrics::WRITE_BUFFER_BYTES;
|
||||
use crate::read::Batch;
|
||||
@@ -51,7 +50,6 @@ use crate::sst::parquet::file_range::PreFilterMode;
|
||||
|
||||
mod builder;
|
||||
pub mod bulk;
|
||||
pub mod partition_tree;
|
||||
pub mod simple_bulk_memtable;
|
||||
mod stats;
|
||||
pub mod time_partition;
|
||||
@@ -70,15 +68,6 @@ pub use time_partition::filter_record_batch;
|
||||
/// Should be unique under the same region.
|
||||
pub type MemtableId = u32;
|
||||
|
||||
/// Config for memtables.
|
||||
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Default)]
|
||||
#[serde(tag = "type", rename_all = "snake_case")]
|
||||
pub enum MemtableConfig {
|
||||
PartitionTree(PartitionTreeConfig),
|
||||
#[default]
|
||||
TimeSeries,
|
||||
}
|
||||
|
||||
/// Options for querying ranges from a memtable.
|
||||
#[derive(Clone)]
|
||||
pub struct RangesOptions {
|
||||
@@ -418,70 +407,80 @@ impl MemtableBuilderProvider {
|
||||
pub(crate) fn builder_for_options(&self, options: &RegionOptions) -> MemtableBuilderRef {
|
||||
let dedup = options.need_dedup();
|
||||
let merge_mode = options.merge_mode();
|
||||
let primary_key_encoding = options.primary_key_encoding();
|
||||
let flat_format = options
|
||||
.sst_format
|
||||
.map(|format| format == FormatType::Flat)
|
||||
.unwrap_or(self.config.default_flat_format);
|
||||
if flat_format {
|
||||
if options.memtable.is_some() {
|
||||
if options.memtable.is_some()
|
||||
&& !matches!(&options.memtable, Some(MemtableOptions::Bulk(_)))
|
||||
{
|
||||
common_telemetry::info!(
|
||||
"Overriding memtable config, use BulkMemtable under flat format"
|
||||
);
|
||||
}
|
||||
|
||||
return Arc::new(
|
||||
BulkMemtableBuilder::new(
|
||||
self.write_buffer_manager.clone(),
|
||||
!dedup, // append_mode: true if not dedup, false if dedup
|
||||
merge_mode,
|
||||
)
|
||||
.with_compact_dispatcher(self.compact_dispatcher.clone()),
|
||||
);
|
||||
return Arc::new(self.bulk_memtable_builder(dedup, merge_mode, options));
|
||||
}
|
||||
|
||||
if primary_key_encoding == PrimaryKeyEncoding::Sparse {
|
||||
if options.memtable.is_some()
|
||||
&& !matches!(&options.memtable, Some(MemtableOptions::Bulk(_)))
|
||||
{
|
||||
common_telemetry::info!(
|
||||
"Overriding memtable config, use BulkMemtable for sparse primary key encoding"
|
||||
);
|
||||
}
|
||||
return Arc::new(self.bulk_memtable_builder(dedup, merge_mode, options));
|
||||
}
|
||||
|
||||
// The format is not flat.
|
||||
match &options.memtable {
|
||||
Some(MemtableOptions::Bulk(config)) => Arc::new(
|
||||
BulkMemtableBuilder::new(self.write_buffer_manager.clone(), !dedup, merge_mode)
|
||||
.with_config(config.clone())
|
||||
.with_compact_dispatcher(self.compact_dispatcher.clone()),
|
||||
),
|
||||
Some(MemtableOptions::TimeSeries) => Arc::new(TimeSeriesMemtableBuilder::new(
|
||||
self.write_buffer_manager.clone(),
|
||||
dedup,
|
||||
merge_mode,
|
||||
)),
|
||||
Some(MemtableOptions::PartitionTree(opts)) => {
|
||||
Arc::new(PartitionTreeMemtableBuilder::new(
|
||||
PartitionTreeConfig {
|
||||
index_max_keys_per_shard: opts.index_max_keys_per_shard,
|
||||
data_freeze_threshold: opts.data_freeze_threshold,
|
||||
fork_dictionary_bytes: opts.fork_dictionary_bytes,
|
||||
dedup,
|
||||
merge_mode,
|
||||
},
|
||||
self.write_buffer_manager.clone(),
|
||||
))
|
||||
}
|
||||
None => self.default_primary_key_memtable_builder(dedup, merge_mode),
|
||||
}
|
||||
}
|
||||
|
||||
fn bulk_memtable_builder(
|
||||
&self,
|
||||
dedup: bool,
|
||||
merge_mode: MergeMode,
|
||||
options: &RegionOptions,
|
||||
) -> BulkMemtableBuilder {
|
||||
let mut builder = BulkMemtableBuilder::new(
|
||||
self.write_buffer_manager.clone(),
|
||||
!dedup, // append_mode: true if not dedup, false if dedup
|
||||
merge_mode,
|
||||
)
|
||||
.with_compact_dispatcher(self.compact_dispatcher.clone());
|
||||
|
||||
if let Some(MemtableOptions::Bulk(config)) = &options.memtable {
|
||||
builder = builder.with_config(config.clone());
|
||||
}
|
||||
|
||||
builder
|
||||
}
|
||||
|
||||
fn default_primary_key_memtable_builder(
|
||||
&self,
|
||||
dedup: bool,
|
||||
merge_mode: MergeMode,
|
||||
) -> MemtableBuilderRef {
|
||||
match &self.config.memtable {
|
||||
MemtableConfig::PartitionTree(config) => {
|
||||
let mut config = config.clone();
|
||||
config.dedup = dedup;
|
||||
Arc::new(PartitionTreeMemtableBuilder::new(
|
||||
config,
|
||||
self.write_buffer_manager.clone(),
|
||||
))
|
||||
}
|
||||
MemtableConfig::TimeSeries => Arc::new(TimeSeriesMemtableBuilder::new(
|
||||
self.write_buffer_manager.clone(),
|
||||
dedup,
|
||||
merge_mode,
|
||||
)),
|
||||
}
|
||||
Arc::new(TimeSeriesMemtableBuilder::new(
|
||||
self.write_buffer_manager.clone(),
|
||||
dedup,
|
||||
merge_mode,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -749,29 +748,9 @@ impl MemtableRange {
|
||||
mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
|
||||
use super::*;
|
||||
use crate::flush::{WriteBufferManager, WriteBufferManagerImpl};
|
||||
|
||||
#[test]
|
||||
fn test_deserialize_memtable_config() {
|
||||
let s = r#"
|
||||
type = "partition_tree"
|
||||
index_max_keys_per_shard = 8192
|
||||
data_freeze_threshold = 1024
|
||||
dedup = true
|
||||
fork_dictionary_bytes = "512MiB"
|
||||
"#;
|
||||
let config: MemtableConfig = toml::from_str(s).unwrap();
|
||||
let MemtableConfig::PartitionTree(memtable_config) = config else {
|
||||
unreachable!()
|
||||
};
|
||||
assert!(memtable_config.dedup);
|
||||
assert_eq!(8192, memtable_config.index_max_keys_per_shard);
|
||||
assert_eq!(1024, memtable_config.data_freeze_threshold);
|
||||
assert_eq!(ReadableSize::mb(512), memtable_config.fork_dictionary_bytes);
|
||||
}
|
||||
use crate::memtable::bulk::BulkMemtableConfig;
|
||||
|
||||
#[test]
|
||||
fn test_alloc_tracker_without_manager() {
|
||||
@@ -824,4 +803,25 @@ fork_dictionary_bytes = "512MiB"
|
||||
assert_eq!(0, manager.memory_usage());
|
||||
assert_eq!(0, manager.mutable_usage());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_forced_bulk_memtable_preserves_bulk_config() {
|
||||
let provider = MemtableBuilderProvider::new(None, Arc::new(MitoConfig::default()));
|
||||
let config = BulkMemtableConfig {
|
||||
merge_threshold: 7,
|
||||
encode_row_threshold: 11,
|
||||
encode_bytes_threshold: 13,
|
||||
max_merge_groups: 17,
|
||||
};
|
||||
let options = RegionOptions {
|
||||
memtable: Some(MemtableOptions::Bulk(config.clone())),
|
||||
primary_key_encoding: Some(PrimaryKeyEncoding::Sparse),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let builder =
|
||||
provider.bulk_memtable_builder(options.need_dedup(), options.merge_mode(), &options);
|
||||
|
||||
assert_eq!(&config, builder.config());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -37,6 +37,8 @@ use common_time::Timestamp;
|
||||
use datatypes::arrow::datatypes::SchemaRef;
|
||||
use mito_codec::key_values::KeyValue;
|
||||
use rayon::prelude::*;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use serde_with::{DisplayFromStr, serde_as};
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::storage::{ColumnId, FileId, RegionId, SequenceRange};
|
||||
use tokio::sync::Semaphore;
|
||||
@@ -98,15 +100,21 @@ static ENCODE_BYTES_THRESHOLD: LazyLock<usize> = LazyLock::new(|| {
|
||||
});
|
||||
|
||||
/// Configuration for bulk memtable.
|
||||
#[derive(Debug, Clone)]
|
||||
#[serde_as]
|
||||
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
|
||||
#[serde(default)]
|
||||
pub struct BulkMemtableConfig {
|
||||
/// Threshold for triggering merge of parts.
|
||||
#[serde_as(as = "DisplayFromStr")]
|
||||
pub merge_threshold: usize,
|
||||
/// Row threshold for encoding parts.
|
||||
#[serde_as(as = "DisplayFromStr")]
|
||||
pub encode_row_threshold: usize,
|
||||
/// Bytes threshold for encoding parts.
|
||||
#[serde_as(as = "DisplayFromStr")]
|
||||
pub encode_bytes_threshold: usize,
|
||||
/// Maximum number of groups for parallel merging.
|
||||
#[serde_as(as = "DisplayFromStr")]
|
||||
pub max_merge_groups: usize,
|
||||
}
|
||||
|
||||
@@ -118,6 +126,16 @@ impl Default for BulkMemtableConfig {
|
||||
encode_bytes_threshold: *ENCODE_BYTES_THRESHOLD,
|
||||
max_merge_groups: *MAX_MERGE_GROUPS,
|
||||
}
|
||||
.sanitize()
|
||||
}
|
||||
}
|
||||
|
||||
impl BulkMemtableConfig {
|
||||
fn sanitize(mut self) -> Self {
|
||||
if self.merge_threshold == 0 {
|
||||
self.merge_threshold = DEFAULT_MERGE_THRESHOLD;
|
||||
}
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
@@ -666,6 +684,7 @@ impl BulkMemtable {
|
||||
append_mode: bool,
|
||||
merge_mode: MergeMode,
|
||||
) -> Self {
|
||||
let config = config.sanitize();
|
||||
let flat_arrow_schema = to_flat_sst_arrow_schema(
|
||||
&metadata,
|
||||
&FlatSchemaOptions::from_encoding(metadata.primary_key_encoding),
|
||||
@@ -1349,11 +1368,22 @@ impl BulkMemtableBuilder {
|
||||
}
|
||||
}
|
||||
|
||||
/// Sets the bulk memtable config.
|
||||
pub fn with_config(mut self, config: BulkMemtableConfig) -> Self {
|
||||
self.config = config;
|
||||
self
|
||||
}
|
||||
|
||||
/// Sets the compact dispatcher.
|
||||
pub fn with_compact_dispatcher(mut self, compact_dispatcher: Arc<CompactDispatcher>) -> Self {
|
||||
self.compact_dispatcher = Some(compact_dispatcher);
|
||||
self
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub(crate) fn config(&self) -> &BulkMemtableConfig {
|
||||
&self.config
|
||||
}
|
||||
}
|
||||
|
||||
impl MemtableBuilder for BulkMemtableBuilder {
|
||||
@@ -1415,6 +1445,24 @@ mod tests {
|
||||
converter.convert()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_bulk_memtable_sanitizes_zero_merge_threshold() {
|
||||
let metadata = metadata_for_test();
|
||||
let config = BulkMemtableConfig {
|
||||
merge_threshold: 0,
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let memtable =
|
||||
BulkMemtable::new(999, config, metadata, None, None, false, MergeMode::LastRow);
|
||||
|
||||
assert_eq!(DEFAULT_MERGE_THRESHOLD, memtable.config.merge_threshold);
|
||||
assert_eq!(
|
||||
DEFAULT_MERGE_THRESHOLD,
|
||||
memtable.compactor.lock().unwrap().config.merge_threshold
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_bulk_memtable_write_read() {
|
||||
let metadata = metadata_for_test();
|
||||
|
||||
@@ -143,11 +143,10 @@ impl BulkIterContext {
|
||||
pub(crate) fn build_pk_filter(&self) -> Option<CachedPrimaryKeyFilter> {
|
||||
let pk_filters = self.pk_filters.as_ref()?;
|
||||
let metadata = self.base.read_format.metadata();
|
||||
// Parquet PK prefilter always supports the partition column.
|
||||
let inner = self
|
||||
.base
|
||||
.codec
|
||||
.primary_key_filter(metadata, Arc::clone(pk_filters), false);
|
||||
.primary_key_filter(metadata, Arc::clone(pk_filters));
|
||||
Some(CachedPrimaryKeyFilter::new(inner))
|
||||
}
|
||||
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@@ -1,221 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::ops::Range;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::memtable::partition_tree::PkId;
|
||||
use crate::memtable::partition_tree::data::DataBatch;
|
||||
use crate::memtable::partition_tree::shard::DataBatchSource;
|
||||
|
||||
/// A reader that dedup sorted batches from a merger.
|
||||
pub struct DedupReader<T> {
|
||||
prev_batch_last_row: Option<(PkId, i64)>,
|
||||
current_batch_range: Option<Range<usize>>,
|
||||
inner: T,
|
||||
}
|
||||
|
||||
impl<T: DataBatchSource> DedupReader<T> {
|
||||
/// Creates a new dedup reader.
|
||||
pub fn try_new(inner: T) -> Result<Self> {
|
||||
let mut res = Self {
|
||||
prev_batch_last_row: None,
|
||||
current_batch_range: None,
|
||||
inner,
|
||||
};
|
||||
res.next()?;
|
||||
Ok(res)
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: DataBatchSource> DataBatchSource for DedupReader<T> {
|
||||
fn is_valid(&self) -> bool {
|
||||
self.current_batch_range.is_some()
|
||||
}
|
||||
|
||||
fn next(&mut self) -> Result<()> {
|
||||
while self.inner.is_valid() {
|
||||
match &mut self.prev_batch_last_row {
|
||||
None => {
|
||||
// First shot, fill prev_batch_last_row and current_batch_range with first batch.
|
||||
let current_batch = self.inner.current_data_batch();
|
||||
let pk_id = self.inner.current_pk_id();
|
||||
let (last_ts, _) = current_batch.last_row();
|
||||
self.prev_batch_last_row = Some((pk_id, last_ts));
|
||||
self.current_batch_range = Some(0..current_batch.num_rows());
|
||||
break;
|
||||
}
|
||||
Some(prev_last_row) => {
|
||||
self.inner.next()?;
|
||||
if !self.inner.is_valid() {
|
||||
// Resets current_batch_range if inner reader is exhausted.
|
||||
self.current_batch_range = None;
|
||||
break;
|
||||
}
|
||||
let current_batch = self.inner.current_data_batch();
|
||||
let current_pk_id = self.inner.current_pk_id();
|
||||
let (first_ts, _) = current_batch.first_row();
|
||||
let rows_in_batch = current_batch.num_rows();
|
||||
|
||||
let (start, end) = if &(current_pk_id, first_ts) == prev_last_row {
|
||||
// First row in this batch duplicated with the last row in previous batch
|
||||
if rows_in_batch == 1 {
|
||||
// If batch is exhausted, move to next batch.
|
||||
continue;
|
||||
} else {
|
||||
// Skip the first row, start from offset 1.
|
||||
(1, rows_in_batch)
|
||||
}
|
||||
} else {
|
||||
// No duplicates found, yield whole batch.
|
||||
(0, rows_in_batch)
|
||||
};
|
||||
|
||||
let (last_ts, _) = current_batch.last_row();
|
||||
*prev_last_row = (current_pk_id, last_ts);
|
||||
self.current_batch_range = Some(start..end);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn current_pk_id(&self) -> PkId {
|
||||
self.inner.current_pk_id()
|
||||
}
|
||||
|
||||
fn current_key(&self) -> Option<&[u8]> {
|
||||
self.inner.current_key()
|
||||
}
|
||||
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
let range = self.current_batch_range.as_ref().unwrap();
|
||||
let data_batch = self.inner.current_data_batch();
|
||||
data_batch.slice(range.start, range.len())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
|
||||
use super::*;
|
||||
use crate::memtable::partition_tree::data::{DataBuffer, DataParts, DataPartsReader};
|
||||
use crate::test_util::memtable_util::{
|
||||
extract_data_batch, metadata_for_test, write_rows_to_buffer,
|
||||
};
|
||||
|
||||
struct MockSource(DataPartsReader);
|
||||
|
||||
impl DataBatchSource for MockSource {
|
||||
fn is_valid(&self) -> bool {
|
||||
self.0.is_valid()
|
||||
}
|
||||
|
||||
fn next(&mut self) -> Result<()> {
|
||||
self.0.next()
|
||||
}
|
||||
|
||||
fn current_pk_id(&self) -> PkId {
|
||||
PkId {
|
||||
shard_id: 0,
|
||||
pk_index: self.0.current_data_batch().pk_index(),
|
||||
}
|
||||
}
|
||||
|
||||
fn current_key(&self) -> Option<&[u8]> {
|
||||
None
|
||||
}
|
||||
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
self.0.current_data_batch()
|
||||
}
|
||||
}
|
||||
|
||||
fn build_data_buffer(
|
||||
meta: RegionMetadataRef,
|
||||
rows: Vec<(u16, Vec<i64>)>,
|
||||
seq: &mut u64,
|
||||
) -> DataBuffer {
|
||||
let mut buffer = DataBuffer::with_capacity(meta.clone(), 10, true);
|
||||
|
||||
for row in rows {
|
||||
let (pk_index, timestamps) = row;
|
||||
let num_rows = timestamps.len() as u64;
|
||||
let v = timestamps.iter().map(|v| Some(*v as f64)).collect();
|
||||
|
||||
write_rows_to_buffer(&mut buffer, &meta, pk_index, timestamps, v, *seq);
|
||||
*seq += num_rows;
|
||||
}
|
||||
buffer
|
||||
}
|
||||
|
||||
fn check_data_parts_reader_dedup(
|
||||
parts: Vec<Vec<(u16, Vec<i64>)>>,
|
||||
expected: Vec<(u16, Vec<(i64, u64)>)>,
|
||||
) {
|
||||
let meta = metadata_for_test();
|
||||
let mut seq = 0;
|
||||
|
||||
let mut frozens = Vec::with_capacity(parts.len());
|
||||
for part in parts {
|
||||
let mut buffer1 = build_data_buffer(meta.clone(), part, &mut seq);
|
||||
let part1 = buffer1.freeze(None, false).unwrap();
|
||||
frozens.push(part1);
|
||||
}
|
||||
|
||||
let parts = DataParts::new(meta, 10, true).with_frozen(frozens);
|
||||
|
||||
let mut res = Vec::with_capacity(expected.len());
|
||||
let mut reader =
|
||||
DedupReader::try_new(MockSource(parts.read().unwrap().build().unwrap())).unwrap();
|
||||
while reader.is_valid() {
|
||||
let batch = reader.current_data_batch();
|
||||
res.push(extract_data_batch(&batch));
|
||||
reader.next().unwrap();
|
||||
}
|
||||
|
||||
assert_eq!(expected, res);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_data_parts_reader_dedup() {
|
||||
check_data_parts_reader_dedup(vec![vec![(0, vec![1, 2])]], vec![(0, vec![(1, 0), (2, 1)])]);
|
||||
|
||||
check_data_parts_reader_dedup(
|
||||
vec![
|
||||
vec![(0, vec![1, 2])],
|
||||
vec![(0, vec![1, 2])],
|
||||
vec![(0, vec![2, 3])],
|
||||
],
|
||||
vec![(0, vec![(1, 2)]), (0, vec![(2, 4)]), (0, vec![(3, 5)])],
|
||||
);
|
||||
|
||||
check_data_parts_reader_dedup(
|
||||
vec![vec![(0, vec![1])], vec![(0, vec![2])], vec![(0, vec![3])]],
|
||||
vec![(0, vec![(1, 0)]), (0, vec![(2, 1)]), (0, vec![(3, 2)])],
|
||||
);
|
||||
|
||||
check_data_parts_reader_dedup(
|
||||
vec![vec![(0, vec![1])], vec![(0, vec![1])], vec![(0, vec![1])]],
|
||||
vec![(0, vec![(1, 2)])],
|
||||
);
|
||||
|
||||
check_data_parts_reader_dedup(
|
||||
vec![vec![(0, vec![1])], vec![(1, vec![1])], vec![(2, vec![1])]],
|
||||
vec![(0, vec![(1, 0)]), (1, vec![(1, 1)]), (2, vec![(1, 2)])],
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,493 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//! Key dictionary of a shard.
|
||||
|
||||
use std::collections::BTreeMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datatypes::arrow::array::{Array, ArrayBuilder, BinaryArray, BinaryBuilder};
|
||||
|
||||
use crate::memtable::partition_tree::PkIndex;
|
||||
use crate::memtable::stats::WriteMetrics;
|
||||
use crate::metrics::MEMTABLE_DICT_BYTES;
|
||||
|
||||
/// Maximum keys in a [DictBlock].
|
||||
const MAX_KEYS_PER_BLOCK: u16 = 256;
|
||||
|
||||
/// The key is mcmp-encoded primary keys, while the values are the pk index and
|
||||
/// optionally sparsely encoded primary keys.
|
||||
type PkIndexMap = BTreeMap<Vec<u8>, (PkIndex, Option<Vec<u8>>)>;
|
||||
|
||||
/// Builder to build a key dictionary.
|
||||
pub struct KeyDictBuilder {
|
||||
/// Max keys of the dictionary.
|
||||
capacity: usize,
|
||||
/// Number of keys in the builder.
|
||||
num_keys: usize,
|
||||
/// Maps primary key to pk index.
|
||||
pk_to_index: PkIndexMap,
|
||||
/// Buffer for active dict block.
|
||||
key_buffer: KeyBuffer,
|
||||
/// Dictionary blocks.
|
||||
dict_blocks: Vec<DictBlock>,
|
||||
/// Bytes allocated by keys in the index.
|
||||
key_bytes_in_index: usize,
|
||||
}
|
||||
|
||||
impl KeyDictBuilder {
|
||||
/// Creates a new builder that can hold up to `capacity` keys.
|
||||
pub fn new(capacity: usize) -> Self {
|
||||
Self {
|
||||
capacity,
|
||||
num_keys: 0,
|
||||
pk_to_index: BTreeMap::new(),
|
||||
key_buffer: KeyBuffer::new(MAX_KEYS_PER_BLOCK.into()),
|
||||
dict_blocks: Vec::with_capacity(capacity / MAX_KEYS_PER_BLOCK as usize + 1),
|
||||
key_bytes_in_index: 0,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the builder is full.
|
||||
pub fn is_full(&self) -> bool {
|
||||
self.num_keys >= self.capacity
|
||||
}
|
||||
|
||||
/// Adds the key to the builder and returns its index if the builder is not full.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the builder is full.
|
||||
pub fn insert_key(
|
||||
&mut self,
|
||||
full_primary_key: &[u8],
|
||||
sparse_key: Option<&[u8]>,
|
||||
metrics: &mut WriteMetrics,
|
||||
) -> PkIndex {
|
||||
assert!(!self.is_full());
|
||||
|
||||
if let Some(pk_index) = self.pk_to_index.get(full_primary_key).map(|v| v.0) {
|
||||
// Already in the builder.
|
||||
return pk_index;
|
||||
}
|
||||
|
||||
if self.key_buffer.len() >= MAX_KEYS_PER_BLOCK.into() {
|
||||
// The write buffer is full. Freeze a dict block.
|
||||
let dict_block = self.key_buffer.finish(false);
|
||||
self.dict_blocks.push(dict_block);
|
||||
}
|
||||
|
||||
// Safety: we have checked the buffer length.
|
||||
let pk_index = self.key_buffer.push_key(full_primary_key);
|
||||
let (sparse_key, sparse_key_len) = if let Some(sparse_key) = sparse_key {
|
||||
(Some(sparse_key.to_vec()), sparse_key.len())
|
||||
} else {
|
||||
(None, 0)
|
||||
};
|
||||
self.pk_to_index
|
||||
.insert(full_primary_key.to_vec(), (pk_index, sparse_key));
|
||||
self.num_keys += 1;
|
||||
|
||||
// Since we store the key twice so the bytes usage doubled.
|
||||
metrics.key_bytes += full_primary_key.len() * 2 + sparse_key_len;
|
||||
self.key_bytes_in_index += full_primary_key.len() + sparse_key_len;
|
||||
|
||||
// Adds key size of index to the metrics.
|
||||
MEMTABLE_DICT_BYTES.add((full_primary_key.len() + sparse_key_len) as i64);
|
||||
|
||||
pk_index
|
||||
}
|
||||
|
||||
/// Memory size of the builder.
|
||||
#[cfg(test)]
|
||||
pub fn memory_size(&self) -> usize {
|
||||
self.key_bytes_in_index
|
||||
+ self.key_buffer.buffer_memory_size()
|
||||
+ self
|
||||
.dict_blocks
|
||||
.iter()
|
||||
.map(|block| block.buffer_memory_size())
|
||||
.sum::<usize>()
|
||||
}
|
||||
|
||||
/// Finishes the builder. The key of the second BTreeMap is sparse-encoded bytes.
|
||||
pub fn finish(&mut self) -> Option<(KeyDict, BTreeMap<Vec<u8>, PkIndex>)> {
|
||||
if self.key_buffer.is_empty() {
|
||||
return None;
|
||||
}
|
||||
let mut pk_to_index_map = BTreeMap::new();
|
||||
|
||||
// Finishes current dict block and resets the pk index.
|
||||
let dict_block = self.key_buffer.finish(true);
|
||||
self.dict_blocks.push(dict_block);
|
||||
// Computes key position and then alter pk index.
|
||||
let mut key_positions = vec![0; self.pk_to_index.len()];
|
||||
|
||||
for (i, (full_pk, (pk_index, sparse_key))) in (std::mem::take(&mut self.pk_to_index))
|
||||
.into_iter()
|
||||
.enumerate()
|
||||
{
|
||||
// The position of the i-th key is the old pk index.
|
||||
key_positions[i] = pk_index;
|
||||
if let Some(sparse_key) = sparse_key {
|
||||
pk_to_index_map.insert(sparse_key, i as PkIndex);
|
||||
}
|
||||
pk_to_index_map.insert(full_pk, i as PkIndex);
|
||||
}
|
||||
|
||||
self.num_keys = 0;
|
||||
let key_bytes_in_index = self.key_bytes_in_index;
|
||||
self.key_bytes_in_index = 0;
|
||||
|
||||
Some((
|
||||
KeyDict {
|
||||
dict_blocks: std::mem::take(&mut self.dict_blocks),
|
||||
key_positions,
|
||||
key_bytes_in_index,
|
||||
},
|
||||
pk_to_index_map,
|
||||
))
|
||||
}
|
||||
|
||||
/// Reads the builder.
|
||||
pub fn read(&self) -> DictBuilderReader {
|
||||
let sorted_pk_indices = self.pk_to_index.values().map(|v| v.0).collect();
|
||||
let block = self.key_buffer.finish_cloned();
|
||||
let mut blocks = Vec::with_capacity(self.dict_blocks.len() + 1);
|
||||
blocks.extend_from_slice(&self.dict_blocks);
|
||||
blocks.push(block);
|
||||
|
||||
DictBuilderReader::new(blocks, sorted_pk_indices)
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for KeyDictBuilder {
|
||||
fn drop(&mut self) {
|
||||
MEMTABLE_DICT_BYTES.sub(self.key_bytes_in_index as i64);
|
||||
}
|
||||
}
|
||||
|
||||
/// Reader to scan the [KeyDictBuilder].
|
||||
#[derive(Default)]
|
||||
pub struct DictBuilderReader {
|
||||
blocks: Vec<DictBlock>,
|
||||
sorted_pk_indices: Vec<PkIndex>,
|
||||
}
|
||||
|
||||
impl DictBuilderReader {
|
||||
fn new(blocks: Vec<DictBlock>, sorted_pk_indices: Vec<PkIndex>) -> Self {
|
||||
Self {
|
||||
blocks,
|
||||
sorted_pk_indices,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the number of keys.
|
||||
#[cfg(test)]
|
||||
pub fn num_keys(&self) -> usize {
|
||||
self.sorted_pk_indices.len()
|
||||
}
|
||||
|
||||
/// Gets the i-th pk index.
|
||||
#[cfg(test)]
|
||||
pub fn pk_index(&self, offset: usize) -> PkIndex {
|
||||
self.sorted_pk_indices[offset]
|
||||
}
|
||||
|
||||
/// Gets the i-th key.
|
||||
#[cfg(test)]
|
||||
pub fn key(&self, offset: usize) -> &[u8] {
|
||||
let pk_index = self.pk_index(offset);
|
||||
self.key_by_pk_index(pk_index)
|
||||
}
|
||||
|
||||
/// Gets the key by the pk index.
|
||||
pub fn key_by_pk_index(&self, pk_index: PkIndex) -> &[u8] {
|
||||
let block_idx = pk_index / MAX_KEYS_PER_BLOCK;
|
||||
self.blocks[block_idx as usize].key_by_pk_index(pk_index)
|
||||
}
|
||||
|
||||
/// Returns pk weights to sort a data part and replaces pk indices.
|
||||
pub(crate) fn pk_weights_to_sort_data(&self, pk_weights: &mut Vec<u16>) {
|
||||
compute_pk_weights(&self.sorted_pk_indices, pk_weights)
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns pk weights to sort a data part and replaces pk indices.
|
||||
fn compute_pk_weights(sorted_pk_indices: &[PkIndex], pk_weights: &mut Vec<u16>) {
|
||||
pk_weights.resize(sorted_pk_indices.len(), 0);
|
||||
for (weight, pk_index) in sorted_pk_indices.iter().enumerate() {
|
||||
pk_weights[*pk_index as usize] = weight as u16;
|
||||
}
|
||||
}
|
||||
|
||||
/// A key dictionary.
|
||||
#[derive(Default)]
|
||||
pub struct KeyDict {
|
||||
// TODO(yingwen): We can use key_positions to do a binary search.
|
||||
/// Unsorted key blocks.
|
||||
dict_blocks: Vec<DictBlock>,
|
||||
/// Maps pk index to position of the key in [Self::dict_blocks].
|
||||
key_positions: Vec<PkIndex>,
|
||||
/// Bytes of keys in the index.
|
||||
key_bytes_in_index: usize,
|
||||
}
|
||||
|
||||
pub type KeyDictRef = Arc<KeyDict>;
|
||||
|
||||
impl KeyDict {
|
||||
/// Gets the primary key by its index.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the index is invalid.
|
||||
pub fn key_by_pk_index(&self, index: PkIndex) -> &[u8] {
|
||||
let position = self.key_positions[index as usize];
|
||||
let block_index = position / MAX_KEYS_PER_BLOCK;
|
||||
self.dict_blocks[block_index as usize].key_by_pk_index(position)
|
||||
}
|
||||
|
||||
/// Returns pk weights to sort a data part and replaces pk indices.
|
||||
pub(crate) fn pk_weights_to_sort_data(&self) -> Vec<u16> {
|
||||
let mut pk_weights = Vec::with_capacity(self.key_positions.len());
|
||||
compute_pk_weights(&self.key_positions, &mut pk_weights);
|
||||
pk_weights
|
||||
}
|
||||
|
||||
/// Returns the shared memory size.
|
||||
pub(crate) fn shared_memory_size(&self) -> usize {
|
||||
self.key_bytes_in_index
|
||||
+ self
|
||||
.dict_blocks
|
||||
.iter()
|
||||
.map(|block| block.buffer_memory_size())
|
||||
.sum::<usize>()
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for KeyDict {
|
||||
fn drop(&mut self) {
|
||||
MEMTABLE_DICT_BYTES.sub(self.key_bytes_in_index as i64);
|
||||
}
|
||||
}
|
||||
|
||||
/// Buffer to store unsorted primary keys.
|
||||
struct KeyBuffer {
|
||||
key_builder: BinaryBuilder,
|
||||
next_pk_index: usize,
|
||||
}
|
||||
|
||||
impl KeyBuffer {
|
||||
fn new(item_capacity: usize) -> Self {
|
||||
Self {
|
||||
key_builder: BinaryBuilder::with_capacity(item_capacity, 0),
|
||||
next_pk_index: 0,
|
||||
}
|
||||
}
|
||||
|
||||
/// Pushes a new key and returns its pk index.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the [PkIndex] type cannot represent the index.
|
||||
fn push_key(&mut self, key: &[u8]) -> PkIndex {
|
||||
let pk_index = self.next_pk_index.try_into().unwrap();
|
||||
self.next_pk_index += 1;
|
||||
self.key_builder.append_value(key);
|
||||
|
||||
pk_index
|
||||
}
|
||||
|
||||
/// Returns number of items in the buffer.
|
||||
fn len(&self) -> usize {
|
||||
self.key_builder.len()
|
||||
}
|
||||
|
||||
/// Returns whether the buffer is empty.
|
||||
fn is_empty(&self) -> bool {
|
||||
self.key_builder.is_empty()
|
||||
}
|
||||
|
||||
/// Returns the buffer size of the builder.
|
||||
#[cfg(test)]
|
||||
fn buffer_memory_size(&self) -> usize {
|
||||
self.key_builder.values_slice().len()
|
||||
+ std::mem::size_of_val(self.key_builder.offsets_slice())
|
||||
+ self
|
||||
.key_builder
|
||||
.validity_slice()
|
||||
.map(|v| v.len())
|
||||
.unwrap_or(0)
|
||||
}
|
||||
|
||||
fn finish(&mut self, reset_index: bool) -> DictBlock {
|
||||
let primary_key = self.key_builder.finish();
|
||||
// Reserve capacity for the new builder. `finish()` the builder will leave the builder
|
||||
// empty with capacity 0.
|
||||
// TODO(yingwen): Do we need to reserve capacity for data?
|
||||
self.key_builder = BinaryBuilder::with_capacity(primary_key.len(), 0);
|
||||
if reset_index {
|
||||
self.next_pk_index = 0;
|
||||
}
|
||||
|
||||
DictBlock::new(primary_key)
|
||||
}
|
||||
|
||||
fn finish_cloned(&self) -> DictBlock {
|
||||
let primary_key = self.key_builder.finish_cloned();
|
||||
|
||||
DictBlock::new(primary_key)
|
||||
}
|
||||
}
|
||||
|
||||
/// A block in the key dictionary.
|
||||
///
|
||||
/// The block is cheap to clone. Keys in the block are unsorted.
|
||||
#[derive(Clone)]
|
||||
struct DictBlock {
|
||||
/// Container of keys in the block.
|
||||
keys: BinaryArray,
|
||||
}
|
||||
|
||||
impl DictBlock {
|
||||
fn new(keys: BinaryArray) -> Self {
|
||||
let buffer_size = keys.get_buffer_memory_size();
|
||||
MEMTABLE_DICT_BYTES.add(buffer_size as i64);
|
||||
|
||||
Self { keys }
|
||||
}
|
||||
|
||||
fn key_by_pk_index(&self, index: PkIndex) -> &[u8] {
|
||||
let pos = index % MAX_KEYS_PER_BLOCK;
|
||||
self.keys.value(pos as usize)
|
||||
}
|
||||
|
||||
fn buffer_memory_size(&self) -> usize {
|
||||
self.keys.get_buffer_memory_size()
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for DictBlock {
|
||||
fn drop(&mut self) {
|
||||
let buffer_size = self.keys.get_buffer_memory_size();
|
||||
MEMTABLE_DICT_BYTES.sub(buffer_size as i64);
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use rand::Rng;
|
||||
|
||||
use super::*;
|
||||
|
||||
fn prepare_input_keys(num_keys: usize) -> Vec<Vec<u8>> {
|
||||
let prefix = ["a", "b", "c", "d", "e", "f"];
|
||||
let mut rng = rand::rng();
|
||||
let mut keys = Vec::with_capacity(num_keys);
|
||||
for i in 0..num_keys {
|
||||
let prefix_idx = rng.random_range(0..prefix.len());
|
||||
// We don't need to decode the primary key in index's test so we format the string
|
||||
// into the key.
|
||||
let key = format!("{}{}", prefix[prefix_idx], i);
|
||||
keys.push(key.into_bytes());
|
||||
}
|
||||
|
||||
keys
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_scan_builder() {
|
||||
let num_keys = MAX_KEYS_PER_BLOCK * 2 + MAX_KEYS_PER_BLOCK / 2;
|
||||
let keys = prepare_input_keys(num_keys.into());
|
||||
|
||||
let mut builder = KeyDictBuilder::new((MAX_KEYS_PER_BLOCK * 3).into());
|
||||
let mut last_pk_index = None;
|
||||
let mut metrics = WriteMetrics::default();
|
||||
for key in &keys {
|
||||
assert!(!builder.is_full());
|
||||
let pk_index = builder.insert_key(key, None, &mut metrics);
|
||||
last_pk_index = Some(pk_index);
|
||||
}
|
||||
assert_eq!(num_keys - 1, last_pk_index.unwrap());
|
||||
let key_bytes: usize = keys.iter().map(|key| key.len() * 2).sum();
|
||||
assert_eq!(key_bytes, metrics.key_bytes);
|
||||
|
||||
let mut expect: Vec<_> = keys
|
||||
.into_iter()
|
||||
.enumerate()
|
||||
.map(|(i, key)| (key, i as PkIndex))
|
||||
.collect();
|
||||
expect.sort_unstable_by(|a, b| a.0.cmp(&b.0));
|
||||
|
||||
let mut result = Vec::with_capacity(expect.len());
|
||||
let reader = builder.read();
|
||||
for i in 0..reader.num_keys() {
|
||||
result.push((reader.key(i).to_vec(), reader.pk_index(i)));
|
||||
}
|
||||
assert_eq!(expect, result);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_dict_memory_size() {
|
||||
let mut builder = KeyDictBuilder::new((MAX_KEYS_PER_BLOCK * 3).into());
|
||||
let mut metrics = WriteMetrics::default();
|
||||
// 513 keys
|
||||
let num_keys = MAX_KEYS_PER_BLOCK * 2 + 1;
|
||||
// Writes 2 blocks
|
||||
for i in 0..num_keys {
|
||||
// Each key is 5 bytes.
|
||||
let key = format!("{i:05}");
|
||||
builder.insert_key(key.as_bytes(), None, &mut metrics);
|
||||
}
|
||||
let key_bytes = num_keys as usize * 5;
|
||||
assert_eq!(key_bytes * 2, metrics.key_bytes);
|
||||
assert_eq!(key_bytes, builder.key_bytes_in_index);
|
||||
assert_eq!(8730, builder.memory_size());
|
||||
|
||||
let (dict, _) = builder.finish().unwrap();
|
||||
assert_eq!(0, builder.key_bytes_in_index);
|
||||
assert_eq!(key_bytes, dict.key_bytes_in_index);
|
||||
assert!(dict.shared_memory_size() > key_bytes);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_builder_finish() {
|
||||
let mut builder = KeyDictBuilder::new((MAX_KEYS_PER_BLOCK * 2).into());
|
||||
let mut metrics = WriteMetrics::default();
|
||||
for i in 0..MAX_KEYS_PER_BLOCK * 2 {
|
||||
let key = format!("{i:010}");
|
||||
assert!(!builder.is_full());
|
||||
builder.insert_key(key.as_bytes(), None, &mut metrics);
|
||||
}
|
||||
assert!(builder.is_full());
|
||||
builder.finish();
|
||||
|
||||
assert!(!builder.is_full());
|
||||
assert_eq!(0, builder.insert_key(b"a0", None, &mut metrics));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_builder_finish_with_sparse_key() {
|
||||
let mut builder = KeyDictBuilder::new((MAX_KEYS_PER_BLOCK * 2).into());
|
||||
let mut metrics = WriteMetrics::default();
|
||||
let full_key = "42".to_string();
|
||||
let sparse_key = &[42u8];
|
||||
|
||||
builder.insert_key(full_key.as_bytes(), Some(sparse_key), &mut metrics);
|
||||
let (dict, pk_to_pk_id) = builder.finish().unwrap();
|
||||
assert_eq!(dict.key_positions.len(), 1);
|
||||
assert_eq!(dict.dict_blocks.len(), 1);
|
||||
assert_eq!(
|
||||
pk_to_pk_id.get(sparse_key.as_slice()),
|
||||
pk_to_pk_id.get(full_key.as_bytes())
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,554 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::cmp::{Ordering, Reverse};
|
||||
use std::collections::BinaryHeap;
|
||||
use std::fmt::Debug;
|
||||
use std::ops::Range;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::memtable::partition_tree::PkIndex;
|
||||
use crate::memtable::partition_tree::data::{DataBatch, DataBufferReader, DataPartReader};
|
||||
|
||||
/// Nodes of merger's heap.
|
||||
pub trait Node: Ord {
|
||||
/// Returns true if current node is not exhausted.
|
||||
fn is_valid(&self) -> bool;
|
||||
|
||||
/// Whether the other node is behind (exclusive) current node.
|
||||
fn is_behind(&self, other: &Self) -> bool;
|
||||
|
||||
/// Advances `len` rows from current batch. If current batch is empty it fetches
|
||||
/// next batch from the node.
|
||||
///
|
||||
/// # Panics
|
||||
/// If the node is invalid.
|
||||
fn advance(&mut self, len: usize) -> Result<()>;
|
||||
|
||||
/// Length of current item.
|
||||
fn current_item_len(&self) -> usize;
|
||||
|
||||
/// Searches first key of `other` in current item and returns the index.
|
||||
fn search_key_in_current_item(&self, other: &Self) -> Result<usize, usize>;
|
||||
}
|
||||
|
||||
pub struct Merger<T: Node> {
|
||||
/// Heap to find node to read.
|
||||
///
|
||||
/// Nodes in the heap are always valid.
|
||||
heap: BinaryHeap<T>,
|
||||
/// Current node to read.
|
||||
///
|
||||
/// The node is always valid if it is not None.
|
||||
current_node: Option<T>,
|
||||
/// The number of rows in current node that are valid to read.
|
||||
current_rows: usize,
|
||||
}
|
||||
|
||||
impl<T: Node> Merger<T> {
|
||||
pub(crate) fn try_new(nodes: Vec<T>) -> Result<Self> {
|
||||
let mut heap = BinaryHeap::with_capacity(nodes.len());
|
||||
for node in nodes {
|
||||
if node.is_valid() {
|
||||
heap.push(node);
|
||||
}
|
||||
}
|
||||
let mut merger = Merger {
|
||||
heap,
|
||||
current_node: None,
|
||||
current_rows: 0,
|
||||
};
|
||||
merger.next()?;
|
||||
Ok(merger)
|
||||
}
|
||||
|
||||
/// Returns true if current merger is still valid.
|
||||
pub(crate) fn is_valid(&self) -> bool {
|
||||
self.current_node.is_some()
|
||||
}
|
||||
|
||||
/// Returns current node to read. Only [Self::current_rows] rows in current node
|
||||
/// are valid to read.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the merger is invalid.
|
||||
pub(crate) fn current_node(&self) -> &T {
|
||||
self.current_node.as_ref().unwrap()
|
||||
}
|
||||
|
||||
/// Returns rows of current node to read.
|
||||
pub(crate) fn current_rows(&self) -> usize {
|
||||
self.current_rows
|
||||
}
|
||||
|
||||
/// Advances the merger to the next item.
|
||||
pub(crate) fn next(&mut self) -> Result<()> {
|
||||
self.maybe_advance_current_node()?;
|
||||
debug_assert!(self.current_node.is_none());
|
||||
|
||||
// Finds node and range to read from the heap.
|
||||
let Some(top_node) = self.heap.pop() else {
|
||||
// Heap is empty.
|
||||
return Ok(());
|
||||
};
|
||||
if let Some(next_node) = self.heap.peek() {
|
||||
if next_node.is_behind(&top_node) {
|
||||
// Does not overlap.
|
||||
self.current_rows = top_node.current_item_len();
|
||||
} else {
|
||||
// Note that the heap ensures the top node always has the minimal row.
|
||||
match top_node.search_key_in_current_item(next_node) {
|
||||
Ok(pos) => {
|
||||
if pos == 0 {
|
||||
// If the first item of top node has duplicate key with the next node,
|
||||
// we can simply return the first row in the top node as it must be the one
|
||||
// with max sequence.
|
||||
self.current_rows = 1;
|
||||
} else {
|
||||
// We don't know which one has the larger sequence so we use the range before
|
||||
// the duplicate pos.
|
||||
self.current_rows = pos;
|
||||
}
|
||||
}
|
||||
Err(pos) => {
|
||||
// No duplication. Output rows before pos.
|
||||
debug_assert!(pos > 0);
|
||||
self.current_rows = pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Top is the only node left. We can read all rows in it.
|
||||
self.current_rows = top_node.current_item_len();
|
||||
}
|
||||
self.current_node = Some(top_node);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn maybe_advance_current_node(&mut self) -> Result<()> {
|
||||
let Some(mut node) = self.current_node.take() else {
|
||||
return Ok(());
|
||||
};
|
||||
|
||||
// Advances current node.
|
||||
node.advance(self.current_rows)?;
|
||||
self.current_rows = 0;
|
||||
if !node.is_valid() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
// Puts the node into the heap.
|
||||
self.heap.push(node);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct DataBatchKey {
|
||||
pub(crate) pk_index: PkIndex,
|
||||
pub(crate) timestamp: i64,
|
||||
}
|
||||
|
||||
pub(crate) enum DataSource {
|
||||
Buffer(DataBufferReader),
|
||||
Part(DataPartReader),
|
||||
}
|
||||
|
||||
impl DataSource {
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
match self {
|
||||
DataSource::Buffer(buffer) => buffer.current_data_batch(),
|
||||
DataSource::Part(p) => p.current_data_batch(),
|
||||
}
|
||||
}
|
||||
|
||||
fn is_valid(&self) -> bool {
|
||||
match self {
|
||||
DataSource::Buffer(b) => b.is_valid(),
|
||||
DataSource::Part(p) => p.is_valid(),
|
||||
}
|
||||
}
|
||||
|
||||
fn next(&mut self) -> Result<()> {
|
||||
match self {
|
||||
DataSource::Buffer(b) => b.next(),
|
||||
DataSource::Part(p) => p.next(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct DataNode {
|
||||
source: DataSource,
|
||||
/// Current range of the batch in the source.
|
||||
current_range: Option<Range<usize>>,
|
||||
}
|
||||
|
||||
impl DataNode {
|
||||
pub(crate) fn new(source: DataSource) -> Self {
|
||||
let current_range = source
|
||||
.is_valid()
|
||||
.then(|| 0..source.current_data_batch().range().len());
|
||||
|
||||
Self {
|
||||
source,
|
||||
current_range,
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
let range = self.current_range();
|
||||
let batch = self.source.current_data_batch();
|
||||
batch.slice(range.start, range.len())
|
||||
}
|
||||
|
||||
fn current_range(&self) -> Range<usize> {
|
||||
self.current_range.clone().unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
impl Ord for DataNode {
|
||||
fn cmp(&self, other: &Self) -> Ordering {
|
||||
let weight = self.current_data_batch().pk_index();
|
||||
let (ts_start, sequence) = self.current_data_batch().first_row();
|
||||
let other_weight = other.current_data_batch().pk_index();
|
||||
let (other_ts_start, other_sequence) = other.current_data_batch().first_row();
|
||||
(weight, ts_start, Reverse(sequence))
|
||||
.cmp(&(other_weight, other_ts_start, Reverse(other_sequence)))
|
||||
.reverse()
|
||||
}
|
||||
}
|
||||
|
||||
impl Eq for DataNode {}
|
||||
|
||||
impl PartialEq<Self> for DataNode {
|
||||
fn eq(&self, other: &Self) -> bool {
|
||||
self.current_data_batch()
|
||||
.first_row()
|
||||
.eq(&other.current_data_batch().first_row())
|
||||
}
|
||||
}
|
||||
|
||||
impl PartialOrd<Self> for DataNode {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
|
||||
Some(self.cmp(other))
|
||||
}
|
||||
}
|
||||
|
||||
impl Node for DataNode {
|
||||
fn is_valid(&self) -> bool {
|
||||
self.current_range.is_some()
|
||||
}
|
||||
|
||||
fn is_behind(&self, other: &Self) -> bool {
|
||||
let pk_weight = self.current_data_batch().pk_index();
|
||||
let (start, seq) = self.current_data_batch().first_row();
|
||||
let other_pk_weight = other.current_data_batch().pk_index();
|
||||
let (other_end, other_seq) = other.current_data_batch().last_row();
|
||||
(pk_weight, start, Reverse(seq)) > (other_pk_weight, other_end, Reverse(other_seq))
|
||||
}
|
||||
|
||||
fn advance(&mut self, len: usize) -> Result<()> {
|
||||
let mut range = self.current_range();
|
||||
debug_assert!(range.len() >= len);
|
||||
|
||||
let remaining = range.len() - len;
|
||||
if remaining == 0 {
|
||||
// Nothing remains, we need to fetch next batch to ensure the current batch is not empty.
|
||||
self.source.next()?;
|
||||
if self.source.is_valid() {
|
||||
self.current_range = Some(0..self.source.current_data_batch().range().len());
|
||||
} else {
|
||||
// The node is exhausted.
|
||||
self.current_range = None;
|
||||
}
|
||||
} else {
|
||||
range.start += len;
|
||||
self.current_range = Some(range);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn current_item_len(&self) -> usize {
|
||||
self.current_range.clone().unwrap().len()
|
||||
}
|
||||
|
||||
fn search_key_in_current_item(&self, other: &Self) -> Result<usize, usize> {
|
||||
let key = other.current_data_batch().first_key();
|
||||
self.current_data_batch().search_key(&key)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use datatypes::arrow::array::UInt64Array;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
|
||||
use super::*;
|
||||
use crate::memtable::partition_tree::data::{DataBuffer, timestamp_array_to_i64_slice};
|
||||
use crate::test_util::memtable_util::{build_key_values_with_ts_seq_values, metadata_for_test};
|
||||
|
||||
fn write_rows_to_buffer(
|
||||
buffer: &mut DataBuffer,
|
||||
schema: &RegionMetadataRef,
|
||||
pk_index: u16,
|
||||
ts: Vec<i64>,
|
||||
sequence: &mut u64,
|
||||
) {
|
||||
let rows = ts.len() as u64;
|
||||
let v0 = ts.iter().map(|v| Some(*v as f64)).collect::<Vec<_>>();
|
||||
let kvs = build_key_values_with_ts_seq_values(
|
||||
schema,
|
||||
"whatever".to_string(),
|
||||
1,
|
||||
ts.into_iter(),
|
||||
v0.into_iter(),
|
||||
*sequence,
|
||||
);
|
||||
|
||||
for kv in kvs.iter() {
|
||||
buffer.write_row(pk_index, &kv);
|
||||
}
|
||||
|
||||
*sequence += rows;
|
||||
}
|
||||
|
||||
fn check_merger_read(nodes: Vec<DataNode>, expected: &[(u16, Vec<(i64, u64)>)]) {
|
||||
let mut merger = Merger::try_new(nodes).unwrap();
|
||||
|
||||
let mut res = vec![];
|
||||
while merger.is_valid() {
|
||||
let data_batch = merger.current_node().current_data_batch();
|
||||
let data_batch = data_batch.slice(0, merger.current_rows());
|
||||
let batch = data_batch.slice_record_batch();
|
||||
let ts_array = batch.column(1);
|
||||
let ts_values: Vec<_> = timestamp_array_to_i64_slice(ts_array).to_vec();
|
||||
let ts_and_seq = ts_values
|
||||
.into_iter()
|
||||
.zip(
|
||||
batch
|
||||
.column(2)
|
||||
.as_any()
|
||||
.downcast_ref::<UInt64Array>()
|
||||
.unwrap()
|
||||
.iter(),
|
||||
)
|
||||
.map(|(ts, seq)| (ts, seq.unwrap()))
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
res.push((data_batch.pk_index(), ts_and_seq));
|
||||
merger.next().unwrap();
|
||||
}
|
||||
assert_eq!(expected, &res);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merger() {
|
||||
let metadata = metadata_for_test();
|
||||
let mut buffer1 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
let weight = &[2, 1, 0];
|
||||
let mut seq = 0;
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 1, vec![2, 3], &mut seq);
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 0, vec![1, 2], &mut seq);
|
||||
let node1 = DataNode::new(DataSource::Part(
|
||||
buffer1.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer2 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 1, vec![3], &mut seq);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 0, vec![1], &mut seq);
|
||||
let node2 = DataNode::new(DataSource::Part(
|
||||
buffer2.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
check_merger_read(
|
||||
vec![node1, node2],
|
||||
&[
|
||||
(1, vec![(2, 0)]),
|
||||
(1, vec![(3, 4)]),
|
||||
(1, vec![(3, 1)]),
|
||||
(2, vec![(1, 5)]),
|
||||
(2, vec![(1, 2), (2, 3)]),
|
||||
],
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merger2() {
|
||||
let metadata = metadata_for_test();
|
||||
let mut buffer1 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
let weight = &[2, 1, 0];
|
||||
let mut seq = 0;
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 1, vec![2, 3], &mut seq);
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 0, vec![1, 2], &mut seq);
|
||||
let node1 = DataNode::new(DataSource::Part(
|
||||
buffer1.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer2 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 1, vec![3], &mut seq);
|
||||
let node2 = DataNode::new(DataSource::Part(
|
||||
buffer2.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer3 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer3, &metadata, 0, vec![2, 3], &mut seq);
|
||||
let node3 = DataNode::new(DataSource::Part(
|
||||
buffer3.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
check_merger_read(
|
||||
vec![node1, node3, node2],
|
||||
&[
|
||||
(1, vec![(2, 0)]),
|
||||
(1, vec![(3, 4)]),
|
||||
(1, vec![(3, 1)]),
|
||||
(2, vec![(1, 2)]),
|
||||
(2, vec![(2, 5)]),
|
||||
(2, vec![(2, 3)]),
|
||||
(2, vec![(3, 6)]),
|
||||
],
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merger_overlapping() {
|
||||
let metadata = metadata_for_test();
|
||||
let mut buffer1 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
let weight = &[0, 1, 2];
|
||||
let mut seq = 0;
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 0, vec![1, 2, 3], &mut seq);
|
||||
let node1 = DataNode::new(DataSource::Part(
|
||||
buffer1.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer2 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 1, vec![2, 3], &mut seq);
|
||||
let node2 = DataNode::new(DataSource::Part(
|
||||
buffer2.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer3 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer3, &metadata, 0, vec![2, 3], &mut seq);
|
||||
let node3 = DataNode::new(DataSource::Part(
|
||||
buffer3.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
check_merger_read(
|
||||
vec![node1, node3, node2],
|
||||
&[
|
||||
(0, vec![(1, 0)]),
|
||||
(0, vec![(2, 5)]),
|
||||
(0, vec![(2, 1)]),
|
||||
(0, vec![(3, 6)]),
|
||||
(0, vec![(3, 2)]),
|
||||
(1, vec![(2, 3), (3, 4)]),
|
||||
],
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merger_parts_and_buffer() {
|
||||
let metadata = metadata_for_test();
|
||||
let mut buffer1 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
let weight = &[0, 1, 2];
|
||||
let mut seq = 0;
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 0, vec![1, 2, 3], &mut seq);
|
||||
let node1 = DataNode::new(DataSource::Buffer(
|
||||
buffer1.read().unwrap().build(Some(weight)).unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer2 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 1, vec![2, 3], &mut seq);
|
||||
let node2 = DataNode::new(DataSource::Part(
|
||||
buffer2.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer3 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer3, &metadata, 0, vec![2, 3], &mut seq);
|
||||
let node3 = DataNode::new(DataSource::Part(
|
||||
buffer3.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
check_merger_read(
|
||||
vec![node1, node3, node2],
|
||||
&[
|
||||
(0, vec![(1, 0)]),
|
||||
(0, vec![(2, 5)]),
|
||||
(0, vec![(2, 1)]),
|
||||
(0, vec![(3, 6)]),
|
||||
(0, vec![(3, 2)]),
|
||||
(1, vec![(2, 3), (3, 4)]),
|
||||
],
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merger_overlapping_2() {
|
||||
let metadata = metadata_for_test();
|
||||
let mut buffer1 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
let weight = &[0, 1, 2];
|
||||
let mut seq = 0;
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 0, vec![1, 2, 2], &mut seq);
|
||||
let node1 = DataNode::new(DataSource::Part(
|
||||
buffer1.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer2 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 0, vec![2], &mut seq);
|
||||
let node2 = DataNode::new(DataSource::Part(
|
||||
buffer2.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer3 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer3, &metadata, 0, vec![2], &mut seq);
|
||||
let node3 = DataNode::new(DataSource::Part(
|
||||
buffer3.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
check_merger_read(
|
||||
vec![node1, node2, node3],
|
||||
&[
|
||||
(0, vec![(1, 0)]),
|
||||
(0, vec![(2, 4)]),
|
||||
(0, vec![(2, 3)]),
|
||||
(0, vec![(2, 2)]),
|
||||
],
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merger_overlapping_3() {
|
||||
let metadata = metadata_for_test();
|
||||
let mut buffer1 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
let weight = &[0, 1, 2];
|
||||
let mut seq = 0;
|
||||
write_rows_to_buffer(&mut buffer1, &metadata, 0, vec![0, 1], &mut seq);
|
||||
let node1 = DataNode::new(DataSource::Part(
|
||||
buffer1.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
let mut buffer2 = DataBuffer::with_capacity(metadata.clone(), 10, true);
|
||||
write_rows_to_buffer(&mut buffer2, &metadata, 0, vec![1], &mut seq);
|
||||
let node2 = DataNode::new(DataSource::Part(
|
||||
buffer2.freeze(Some(weight), true).unwrap().read().unwrap(),
|
||||
));
|
||||
|
||||
check_merger_read(
|
||||
vec![node1, node2],
|
||||
&[(0, vec![(0, 0)]), (0, vec![(1, 2)]), (0, vec![(1, 1)])],
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -1,590 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//! Partition of a partition tree.
|
||||
//!
|
||||
//! We only support partitioning the tree by pre-defined internal columns.
|
||||
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::sync::{Arc, RwLock};
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use api::v1::SemanticType;
|
||||
use common_recordbatch::filter::SimpleFilterEvaluator;
|
||||
use mito_codec::key_values::KeyValue;
|
||||
use mito_codec::primary_key_filter::is_partition_column;
|
||||
use mito_codec::row_converter::{PrimaryKeyCodec, PrimaryKeyFilter};
|
||||
use snafu::ResultExt;
|
||||
use store_api::codec::PrimaryKeyEncoding;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME;
|
||||
use store_api::storage::ColumnId;
|
||||
|
||||
use crate::error::{EncodeSnafu, Result};
|
||||
use crate::memtable::partition_tree::data::{DATA_INIT_CAP, DataBatch, DataParts};
|
||||
use crate::memtable::partition_tree::dedup::DedupReader;
|
||||
use crate::memtable::partition_tree::shard::{
|
||||
BoxedDataBatchSource, Shard, ShardMerger, ShardNode, ShardSource,
|
||||
};
|
||||
use crate::memtable::partition_tree::shard_builder::ShardBuilder;
|
||||
use crate::memtable::partition_tree::{PartitionTreeConfig, PkId};
|
||||
use crate::memtable::stats::WriteMetrics;
|
||||
use crate::metrics::PARTITION_TREE_READ_STAGE_ELAPSED;
|
||||
use crate::read::{Batch, BatchBuilder};
|
||||
|
||||
/// Key of a partition.
|
||||
pub type PartitionKey = u32;
|
||||
|
||||
/// A tree partition.
|
||||
pub struct Partition {
|
||||
inner: RwLock<Inner>,
|
||||
/// Whether to dedup batches.
|
||||
dedup: bool,
|
||||
}
|
||||
|
||||
pub type PartitionRef = Arc<Partition>;
|
||||
|
||||
impl Partition {
|
||||
/// Creates a new partition.
|
||||
pub fn new(metadata: RegionMetadataRef, config: &PartitionTreeConfig) -> Self {
|
||||
Partition {
|
||||
inner: RwLock::new(Inner::new(metadata, config)),
|
||||
dedup: config.dedup,
|
||||
}
|
||||
}
|
||||
|
||||
/// Writes to the partition with a primary key.
|
||||
pub fn write_with_key(
|
||||
&self,
|
||||
primary_key: &mut Vec<u8>,
|
||||
row_codec: &dyn PrimaryKeyCodec,
|
||||
key_value: KeyValue,
|
||||
re_encode: bool,
|
||||
metrics: &mut WriteMetrics,
|
||||
) -> Result<()> {
|
||||
let mut inner = self.inner.write().unwrap();
|
||||
// Freeze the shard builder if needed.
|
||||
if inner.shard_builder.should_freeze() {
|
||||
inner.freeze_active_shard()?;
|
||||
}
|
||||
|
||||
// Finds key in shards, now we ensure one key only exists in one shard.
|
||||
if let Some(pk_id) = inner.find_key_in_shards(primary_key) {
|
||||
inner.write_to_shard(pk_id, &key_value)?;
|
||||
inner.num_rows += 1;
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
// Key does not yet exist in shard or builder, encode and insert the full primary key.
|
||||
if re_encode {
|
||||
match row_codec.encoding() {
|
||||
PrimaryKeyEncoding::Dense => {
|
||||
// `primary_key` is sparse, re-encode the full primary key.
|
||||
let sparse_key = primary_key.clone();
|
||||
primary_key.clear();
|
||||
row_codec
|
||||
.encode_key_value(&key_value, primary_key)
|
||||
.context(EncodeSnafu)?;
|
||||
let pk_id = inner.shard_builder.write_with_key(
|
||||
primary_key,
|
||||
Some(&sparse_key),
|
||||
&key_value,
|
||||
metrics,
|
||||
);
|
||||
inner.pk_to_pk_id.insert(sparse_key, pk_id);
|
||||
}
|
||||
PrimaryKeyEncoding::Sparse => {
|
||||
let sparse_key = primary_key.clone();
|
||||
let pk_id = inner.shard_builder.write_with_key(
|
||||
primary_key,
|
||||
Some(&sparse_key),
|
||||
&key_value,
|
||||
metrics,
|
||||
);
|
||||
inner.pk_to_pk_id.insert(sparse_key, pk_id);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// `primary_key` is already the full primary key.
|
||||
let pk_id = inner
|
||||
.shard_builder
|
||||
.write_with_key(primary_key, None, &key_value, metrics);
|
||||
inner.pk_to_pk_id.insert(std::mem::take(primary_key), pk_id);
|
||||
};
|
||||
|
||||
inner.num_rows += 1;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Writes to the partition without a primary key.
|
||||
pub fn write_no_key(&self, key_value: KeyValue) -> Result<()> {
|
||||
let mut inner = self.inner.write().unwrap();
|
||||
// If no primary key, always write to the first shard.
|
||||
debug_assert!(!inner.shards.is_empty());
|
||||
debug_assert_eq!(1, inner.shard_builder.current_shard_id());
|
||||
|
||||
// A dummy pk id.
|
||||
let pk_id = PkId {
|
||||
shard_id: 0,
|
||||
pk_index: 0,
|
||||
};
|
||||
inner.shards[0].write_with_pk_id(pk_id, &key_value)?;
|
||||
inner.num_rows += 1;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn build_primary_key_filter(
|
||||
need_prune_key: bool,
|
||||
metadata: &RegionMetadataRef,
|
||||
row_codec: &dyn PrimaryKeyCodec,
|
||||
filters: &Arc<Vec<SimpleFilterEvaluator>>,
|
||||
) -> Option<Box<dyn PrimaryKeyFilter>> {
|
||||
if need_prune_key {
|
||||
// TODO(yingwen): Remove `skip_partition_column` after dropping PartitionTreeMemtable.
|
||||
let filter = row_codec.primary_key_filter(metadata, filters.clone(), true);
|
||||
Some(filter)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
/// Scans data in the partition.
|
||||
pub fn read(&self, mut context: ReadPartitionContext) -> Result<PartitionReader> {
|
||||
let start = Instant::now();
|
||||
let (builder_source, shard_reader_builders) = {
|
||||
let inner = self.inner.read().unwrap();
|
||||
let mut shard_source = Vec::with_capacity(inner.shards.len() + 1);
|
||||
let builder_reader = if !inner.shard_builder.is_empty() {
|
||||
let builder_reader = inner.shard_builder.read(&mut context.pk_weights)?;
|
||||
Some(builder_reader)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
for shard in &inner.shards {
|
||||
if !shard.is_empty() {
|
||||
let shard_reader_builder = shard.read()?;
|
||||
shard_source.push(shard_reader_builder);
|
||||
}
|
||||
}
|
||||
(builder_reader, shard_source)
|
||||
};
|
||||
|
||||
context.metrics.num_shards += shard_reader_builders.len();
|
||||
|
||||
let mut nodes = shard_reader_builders
|
||||
.into_iter()
|
||||
.map(|builder| {
|
||||
let primary_key_filter = Self::build_primary_key_filter(
|
||||
context.need_prune_key,
|
||||
&context.metadata,
|
||||
context.row_codec.as_ref(),
|
||||
&context.filters,
|
||||
);
|
||||
Ok(ShardNode::new(ShardSource::Shard(
|
||||
builder.build(primary_key_filter)?,
|
||||
)))
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()?;
|
||||
|
||||
if let Some(builder) = builder_source {
|
||||
context.metrics.num_builder += 1;
|
||||
let primary_key_filter = Self::build_primary_key_filter(
|
||||
context.need_prune_key,
|
||||
&context.metadata,
|
||||
context.row_codec.as_ref(),
|
||||
&context.filters,
|
||||
);
|
||||
// Move the initialization of ShardBuilderReader out of read lock.
|
||||
let shard_builder_reader =
|
||||
builder.build(Some(&context.pk_weights), primary_key_filter)?;
|
||||
nodes.push(ShardNode::new(ShardSource::Builder(shard_builder_reader)));
|
||||
}
|
||||
|
||||
// Creating a shard merger will invoke next so we do it outside the lock.
|
||||
let merger = ShardMerger::try_new(nodes)?;
|
||||
if self.dedup {
|
||||
let source = DedupReader::try_new(merger)?;
|
||||
context.metrics.build_partition_reader += start.elapsed();
|
||||
PartitionReader::new(context, Box::new(source))
|
||||
} else {
|
||||
context.metrics.build_partition_reader += start.elapsed();
|
||||
PartitionReader::new(context, Box::new(merger))
|
||||
}
|
||||
}
|
||||
|
||||
/// Freezes the partition.
|
||||
pub fn freeze(&self) -> Result<()> {
|
||||
let mut inner = self.inner.write().unwrap();
|
||||
inner.freeze_active_shard()?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Forks the partition.
|
||||
///
|
||||
/// Must freeze the partition before fork.
|
||||
pub fn fork(&self, metadata: &RegionMetadataRef, config: &PartitionTreeConfig) -> Partition {
|
||||
let (shards, shard_builder) = {
|
||||
let inner = self.inner.read().unwrap();
|
||||
debug_assert!(inner.shard_builder.is_empty());
|
||||
let shard_builder = ShardBuilder::new(
|
||||
metadata.clone(),
|
||||
config,
|
||||
inner.shard_builder.current_shard_id(),
|
||||
);
|
||||
let shards = inner
|
||||
.shards
|
||||
.iter()
|
||||
.map(|shard| shard.fork(metadata.clone()))
|
||||
.collect();
|
||||
|
||||
(shards, shard_builder)
|
||||
};
|
||||
let pk_to_pk_id = {
|
||||
let mut inner = self.inner.write().unwrap();
|
||||
std::mem::take(&mut inner.pk_to_pk_id)
|
||||
};
|
||||
|
||||
Partition {
|
||||
inner: RwLock::new(Inner {
|
||||
metadata: metadata.clone(),
|
||||
shard_builder,
|
||||
shards,
|
||||
num_rows: 0,
|
||||
pk_to_pk_id,
|
||||
frozen: false,
|
||||
}),
|
||||
dedup: self.dedup,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the partition has data.
|
||||
pub fn has_data(&self) -> bool {
|
||||
let inner = self.inner.read().unwrap();
|
||||
inner.num_rows > 0
|
||||
}
|
||||
|
||||
/// Gets the stats of the partition.
|
||||
pub(crate) fn stats(&self) -> PartitionStats {
|
||||
let inner = self.inner.read().unwrap();
|
||||
let num_rows = inner.num_rows;
|
||||
let shard_num = inner.shards.len();
|
||||
let shared_memory_size = inner
|
||||
.shards
|
||||
.iter()
|
||||
.map(|shard| shard.shared_memory_size())
|
||||
.sum();
|
||||
PartitionStats {
|
||||
num_rows,
|
||||
shard_num,
|
||||
shared_memory_size,
|
||||
}
|
||||
}
|
||||
|
||||
/// Get partition key from the key value.
|
||||
pub(crate) fn get_partition_key(key_value: &KeyValue, is_partitioned: bool) -> PartitionKey {
|
||||
if !is_partitioned {
|
||||
return PartitionKey::default();
|
||||
}
|
||||
|
||||
key_value.partition_key()
|
||||
}
|
||||
|
||||
/// Returns true if the region can be partitioned.
|
||||
pub(crate) fn has_multi_partitions(metadata: &RegionMetadataRef) -> bool {
|
||||
metadata
|
||||
.primary_key_columns()
|
||||
.next()
|
||||
.map(|meta| meta.column_schema.name == DATA_SCHEMA_TABLE_ID_COLUMN_NAME)
|
||||
.unwrap_or(false)
|
||||
}
|
||||
|
||||
pub(crate) fn series_count(&self) -> usize {
|
||||
self.inner.read().unwrap().series_count()
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct PartitionStats {
|
||||
pub(crate) num_rows: usize,
|
||||
pub(crate) shard_num: usize,
|
||||
pub(crate) shared_memory_size: usize,
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
struct PartitionReaderMetrics {
|
||||
build_partition_reader: Duration,
|
||||
read_source: Duration,
|
||||
data_batch_to_batch: Duration,
|
||||
num_builder: usize,
|
||||
num_shards: usize,
|
||||
}
|
||||
|
||||
/// Reader to scan rows in a partition.
|
||||
///
|
||||
/// It can merge rows from multiple shards.
|
||||
pub struct PartitionReader {
|
||||
context: ReadPartitionContext,
|
||||
source: BoxedDataBatchSource,
|
||||
}
|
||||
|
||||
impl PartitionReader {
|
||||
fn new(context: ReadPartitionContext, source: BoxedDataBatchSource) -> Result<Self> {
|
||||
let reader = Self { context, source };
|
||||
|
||||
Ok(reader)
|
||||
}
|
||||
|
||||
/// Returns true if the reader is valid.
|
||||
pub fn is_valid(&self) -> bool {
|
||||
self.source.is_valid()
|
||||
}
|
||||
|
||||
/// Advances the reader.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the reader is invalid.
|
||||
pub fn next(&mut self) -> Result<()> {
|
||||
self.advance_source()
|
||||
}
|
||||
|
||||
/// Converts current data batch into a [Batch].
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the reader is invalid.
|
||||
pub fn convert_current_batch(&mut self) -> Result<Batch> {
|
||||
let start = Instant::now();
|
||||
let data_batch = self.source.current_data_batch();
|
||||
let batch = data_batch_to_batch(
|
||||
&self.context.metadata,
|
||||
&self.context.projection,
|
||||
self.source.current_key(),
|
||||
data_batch,
|
||||
)?;
|
||||
self.context.metrics.data_batch_to_batch += start.elapsed();
|
||||
Ok(batch)
|
||||
}
|
||||
|
||||
pub(crate) fn into_context(self) -> ReadPartitionContext {
|
||||
self.context
|
||||
}
|
||||
|
||||
fn advance_source(&mut self) -> Result<()> {
|
||||
let read_source = Instant::now();
|
||||
self.source.next()?;
|
||||
self.context.metrics.read_source += read_source.elapsed();
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
/// Structs to reuse across readers to avoid allocating for each reader.
|
||||
pub(crate) struct ReadPartitionContext {
|
||||
metadata: RegionMetadataRef,
|
||||
row_codec: Arc<dyn PrimaryKeyCodec>,
|
||||
projection: HashSet<ColumnId>,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
/// Buffer to store pk weights.
|
||||
pk_weights: Vec<u16>,
|
||||
need_prune_key: bool,
|
||||
metrics: PartitionReaderMetrics,
|
||||
}
|
||||
|
||||
impl Drop for ReadPartitionContext {
|
||||
fn drop(&mut self) {
|
||||
let partition_read_source = self.metrics.read_source.as_secs_f64();
|
||||
PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["partition_read_source"])
|
||||
.observe(partition_read_source);
|
||||
let partition_data_batch_to_batch = self.metrics.data_batch_to_batch.as_secs_f64();
|
||||
PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["partition_data_batch_to_batch"])
|
||||
.observe(partition_data_batch_to_batch);
|
||||
|
||||
common_telemetry::debug!(
|
||||
"TreeIter partitions metrics, \
|
||||
num_builder: {}, \
|
||||
num_shards: {}, \
|
||||
build_partition_reader: {}s, \
|
||||
partition_read_source: {}s, \
|
||||
partition_data_batch_to_batch: {}s",
|
||||
self.metrics.num_builder,
|
||||
self.metrics.num_shards,
|
||||
self.metrics.build_partition_reader.as_secs_f64(),
|
||||
partition_read_source,
|
||||
partition_data_batch_to_batch,
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
impl ReadPartitionContext {
|
||||
pub(crate) fn new(
|
||||
metadata: RegionMetadataRef,
|
||||
row_codec: Arc<dyn PrimaryKeyCodec>,
|
||||
projection: HashSet<ColumnId>,
|
||||
filters: Arc<Vec<SimpleFilterEvaluator>>,
|
||||
) -> ReadPartitionContext {
|
||||
let need_prune_key = Self::need_prune_key(&metadata, &filters);
|
||||
ReadPartitionContext {
|
||||
metadata,
|
||||
row_codec,
|
||||
projection,
|
||||
filters,
|
||||
pk_weights: Vec::new(),
|
||||
need_prune_key,
|
||||
metrics: Default::default(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Does filter contain predicate on primary key columns after pruning the
|
||||
/// partition column.
|
||||
fn need_prune_key(metadata: &RegionMetadataRef, filters: &[SimpleFilterEvaluator]) -> bool {
|
||||
for filter in filters {
|
||||
// We already pruned partitions before so we skip the partition column.
|
||||
if is_partition_column(filter.column_name()) {
|
||||
continue;
|
||||
}
|
||||
let Some(column) = metadata.column_by_name(filter.column_name()) else {
|
||||
continue;
|
||||
};
|
||||
if column.semantic_type != SemanticType::Tag {
|
||||
continue;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
// TODO(yingwen): Pushdown projection to shard readers.
|
||||
/// Converts a [DataBatch] to a [Batch].
|
||||
fn data_batch_to_batch(
|
||||
metadata: &RegionMetadataRef,
|
||||
projection: &HashSet<ColumnId>,
|
||||
key: Option<&[u8]>,
|
||||
data_batch: DataBatch,
|
||||
) -> Result<Batch> {
|
||||
let record_batch = data_batch.slice_record_batch();
|
||||
let primary_key = key.map(|k| k.to_vec()).unwrap_or_default();
|
||||
let mut builder = BatchBuilder::new(primary_key);
|
||||
builder
|
||||
.timestamps_array(record_batch.column(1).clone())?
|
||||
.sequences_array(record_batch.column(2).clone())?
|
||||
.op_types_array(record_batch.column(3).clone())?;
|
||||
|
||||
if record_batch.num_columns() <= 4 {
|
||||
// No fields.
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
// Iterate all field columns.
|
||||
for (array, field) in record_batch
|
||||
.columns()
|
||||
.iter()
|
||||
.zip(record_batch.schema().fields().iter())
|
||||
.skip(4)
|
||||
{
|
||||
// TODO(yingwen): Avoid finding column by name. We know the schema of a DataBatch.
|
||||
// Safety: metadata should contain all fields.
|
||||
let column_id = metadata.column_by_name(field.name()).unwrap().column_id;
|
||||
if !projection.contains(&column_id) {
|
||||
continue;
|
||||
}
|
||||
builder.push_field_array(column_id, array.clone())?;
|
||||
}
|
||||
|
||||
builder.build()
|
||||
}
|
||||
|
||||
/// Inner struct of the partition.
|
||||
///
|
||||
/// A key only exists in one shard.
|
||||
struct Inner {
|
||||
metadata: RegionMetadataRef,
|
||||
/// Map to index pk to pk id.
|
||||
pk_to_pk_id: HashMap<Vec<u8>, PkId>,
|
||||
/// Shard whose dictionary is active.
|
||||
shard_builder: ShardBuilder,
|
||||
/// Shards with frozen dictionary.
|
||||
shards: Vec<Shard>,
|
||||
num_rows: usize,
|
||||
frozen: bool,
|
||||
}
|
||||
|
||||
impl Inner {
|
||||
fn new(metadata: RegionMetadataRef, config: &PartitionTreeConfig) -> Self {
|
||||
let (shards, current_shard_id) = if metadata.primary_key.is_empty() {
|
||||
let data_parts = DataParts::new(metadata.clone(), DATA_INIT_CAP, config.dedup);
|
||||
(
|
||||
vec![Shard::new(
|
||||
0,
|
||||
None,
|
||||
data_parts,
|
||||
config.dedup,
|
||||
config.data_freeze_threshold,
|
||||
)],
|
||||
1,
|
||||
)
|
||||
} else {
|
||||
(Vec::new(), 0)
|
||||
};
|
||||
let shard_builder = ShardBuilder::new(metadata.clone(), config, current_shard_id);
|
||||
Self {
|
||||
metadata,
|
||||
pk_to_pk_id: HashMap::new(),
|
||||
shard_builder,
|
||||
shards,
|
||||
num_rows: 0,
|
||||
frozen: false,
|
||||
}
|
||||
}
|
||||
|
||||
fn find_key_in_shards(&self, primary_key: &[u8]) -> Option<PkId> {
|
||||
assert!(!self.frozen);
|
||||
self.pk_to_pk_id.get(primary_key).copied()
|
||||
}
|
||||
|
||||
fn write_to_shard(&mut self, pk_id: PkId, key_value: &KeyValue) -> Result<()> {
|
||||
if pk_id.shard_id == self.shard_builder.current_shard_id() {
|
||||
self.shard_builder.write_with_pk_id(pk_id, key_value);
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
// Safety: We find the shard by shard id.
|
||||
let shard = self
|
||||
.shards
|
||||
.iter_mut()
|
||||
.find(|shard| shard.shard_id == pk_id.shard_id)
|
||||
.unwrap();
|
||||
shard.write_with_pk_id(pk_id, key_value)?;
|
||||
self.num_rows += 1;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn freeze_active_shard(&mut self) -> Result<()> {
|
||||
if let Some(shard) = self
|
||||
.shard_builder
|
||||
.finish(self.metadata.clone(), &mut self.pk_to_pk_id)?
|
||||
{
|
||||
self.shards.push(shard);
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Returns count of timeseries.
|
||||
fn series_count(&self) -> usize {
|
||||
self.pk_to_pk_id.len()
|
||||
}
|
||||
}
|
||||
@@ -1,580 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//! Shard in a partition.
|
||||
|
||||
use std::cmp::Ordering;
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use mito_codec::key_values::KeyValue;
|
||||
use mito_codec::row_converter::PrimaryKeyFilter;
|
||||
use snafu::ResultExt;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
|
||||
use crate::error::{DecodeSnafu, Result};
|
||||
use crate::memtable::partition_tree::data::{
|
||||
DATA_INIT_CAP, DataBatch, DataParts, DataPartsReader, DataPartsReaderBuilder,
|
||||
};
|
||||
use crate::memtable::partition_tree::dict::KeyDictRef;
|
||||
use crate::memtable::partition_tree::merger::{Merger, Node};
|
||||
use crate::memtable::partition_tree::shard_builder::ShardBuilderReader;
|
||||
use crate::memtable::partition_tree::{PkId, PkIndex, ShardId};
|
||||
use crate::metrics::PARTITION_TREE_READ_STAGE_ELAPSED;
|
||||
|
||||
/// Shard stores data related to the same key dictionary.
|
||||
pub struct Shard {
|
||||
pub(crate) shard_id: ShardId,
|
||||
/// Key dictionary of the shard. `None` if the schema of the tree doesn't have a primary key.
|
||||
key_dict: Option<KeyDictRef>,
|
||||
/// Data in the shard.
|
||||
data_parts: DataParts,
|
||||
dedup: bool,
|
||||
/// Number of rows to freeze a data part.
|
||||
data_freeze_threshold: usize,
|
||||
}
|
||||
|
||||
impl Shard {
|
||||
/// Returns a new shard.
|
||||
pub fn new(
|
||||
shard_id: ShardId,
|
||||
key_dict: Option<KeyDictRef>,
|
||||
data_parts: DataParts,
|
||||
dedup: bool,
|
||||
data_freeze_threshold: usize,
|
||||
) -> Shard {
|
||||
Shard {
|
||||
shard_id,
|
||||
key_dict,
|
||||
data_parts,
|
||||
dedup,
|
||||
data_freeze_threshold,
|
||||
}
|
||||
}
|
||||
|
||||
/// Writes a key value into the shard.
|
||||
///
|
||||
/// It will freezes the active buffer if it is full.
|
||||
pub fn write_with_pk_id(&mut self, pk_id: PkId, key_value: &KeyValue) -> Result<()> {
|
||||
debug_assert_eq!(self.shard_id, pk_id.shard_id);
|
||||
|
||||
if self.data_parts.num_active_rows() >= self.data_freeze_threshold {
|
||||
self.data_parts.freeze()?;
|
||||
}
|
||||
|
||||
self.data_parts.write_row(pk_id.pk_index, key_value);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Scans the shard.
|
||||
// TODO(yingwen): Push down projection to data parts.
|
||||
pub fn read(&self) -> Result<ShardReaderBuilder> {
|
||||
let parts_reader = self.data_parts.read()?;
|
||||
|
||||
Ok(ShardReaderBuilder {
|
||||
shard_id: self.shard_id,
|
||||
key_dict: self.key_dict.clone(),
|
||||
inner: parts_reader,
|
||||
})
|
||||
}
|
||||
|
||||
/// Forks a shard.
|
||||
pub fn fork(&self, metadata: RegionMetadataRef) -> Shard {
|
||||
Shard {
|
||||
shard_id: self.shard_id,
|
||||
key_dict: self.key_dict.clone(),
|
||||
data_parts: DataParts::new(metadata, DATA_INIT_CAP, self.dedup),
|
||||
dedup: self.dedup,
|
||||
data_freeze_threshold: self.data_freeze_threshold,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the shard is empty (No data).
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.data_parts.is_empty()
|
||||
}
|
||||
|
||||
/// Returns the memory size of the shard part.
|
||||
pub(crate) fn shared_memory_size(&self) -> usize {
|
||||
self.key_dict
|
||||
.as_ref()
|
||||
.map(|dict| dict.shared_memory_size())
|
||||
.unwrap_or(0)
|
||||
}
|
||||
}
|
||||
|
||||
/// Source that returns [DataBatch].
|
||||
pub trait DataBatchSource {
|
||||
/// Returns whether current source is still valid.
|
||||
fn is_valid(&self) -> bool;
|
||||
|
||||
/// Advances source to next data batch.
|
||||
fn next(&mut self) -> Result<()>;
|
||||
|
||||
/// Returns current pk id.
|
||||
/// # Panics
|
||||
/// If source is not valid.
|
||||
fn current_pk_id(&self) -> PkId;
|
||||
|
||||
/// Returns the current primary key bytes or None if it doesn't have primary key.
|
||||
///
|
||||
/// # Panics
|
||||
/// If source is not valid.
|
||||
fn current_key(&self) -> Option<&[u8]>;
|
||||
|
||||
/// Returns the data part.
|
||||
/// # Panics
|
||||
/// If source is not valid.
|
||||
fn current_data_batch(&self) -> DataBatch<'_>;
|
||||
}
|
||||
|
||||
pub type BoxedDataBatchSource = Box<dyn DataBatchSource + Send>;
|
||||
|
||||
pub struct ShardReaderBuilder {
|
||||
shard_id: ShardId,
|
||||
key_dict: Option<KeyDictRef>,
|
||||
inner: DataPartsReaderBuilder,
|
||||
}
|
||||
|
||||
impl ShardReaderBuilder {
|
||||
pub(crate) fn build(
|
||||
self,
|
||||
key_filter: Option<Box<dyn PrimaryKeyFilter>>,
|
||||
) -> Result<ShardReader> {
|
||||
let ShardReaderBuilder {
|
||||
shard_id,
|
||||
key_dict,
|
||||
inner,
|
||||
} = self;
|
||||
let now = Instant::now();
|
||||
let parts_reader = inner.build()?;
|
||||
ShardReader::new(shard_id, key_dict, parts_reader, key_filter, now.elapsed())
|
||||
}
|
||||
}
|
||||
|
||||
/// Reader to read rows in a shard.
|
||||
pub struct ShardReader {
|
||||
shard_id: ShardId,
|
||||
key_dict: Option<KeyDictRef>,
|
||||
parts_reader: DataPartsReader,
|
||||
key_filter: Option<Box<dyn PrimaryKeyFilter>>,
|
||||
last_yield_pk_index: Option<PkIndex>,
|
||||
keys_before_pruning: usize,
|
||||
keys_after_pruning: usize,
|
||||
prune_pk_cost: Duration,
|
||||
data_build_cost: Duration,
|
||||
}
|
||||
|
||||
impl ShardReader {
|
||||
fn new(
|
||||
shard_id: ShardId,
|
||||
key_dict: Option<KeyDictRef>,
|
||||
parts_reader: DataPartsReader,
|
||||
key_filter: Option<Box<dyn PrimaryKeyFilter>>,
|
||||
data_build_cost: Duration,
|
||||
) -> Result<Self> {
|
||||
let has_pk = key_dict.is_some();
|
||||
let mut reader = Self {
|
||||
shard_id,
|
||||
key_dict,
|
||||
parts_reader,
|
||||
key_filter: if has_pk { key_filter } else { None },
|
||||
last_yield_pk_index: None,
|
||||
keys_before_pruning: 0,
|
||||
keys_after_pruning: 0,
|
||||
prune_pk_cost: Duration::default(),
|
||||
data_build_cost,
|
||||
};
|
||||
reader.prune_batch_by_key()?;
|
||||
|
||||
Ok(reader)
|
||||
}
|
||||
|
||||
fn is_valid(&self) -> bool {
|
||||
self.parts_reader.is_valid()
|
||||
}
|
||||
|
||||
fn next(&mut self) -> Result<()> {
|
||||
self.parts_reader.next()?;
|
||||
self.prune_batch_by_key()
|
||||
}
|
||||
|
||||
fn current_key(&self) -> Option<&[u8]> {
|
||||
let pk_index = self.parts_reader.current_data_batch().pk_index();
|
||||
self.key_dict
|
||||
.as_ref()
|
||||
.map(|dict| dict.key_by_pk_index(pk_index))
|
||||
}
|
||||
|
||||
fn current_pk_id(&self) -> PkId {
|
||||
let pk_index = self.parts_reader.current_data_batch().pk_index();
|
||||
PkId {
|
||||
shard_id: self.shard_id,
|
||||
pk_index,
|
||||
}
|
||||
}
|
||||
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
self.parts_reader.current_data_batch()
|
||||
}
|
||||
|
||||
fn prune_batch_by_key(&mut self) -> Result<()> {
|
||||
let Some(key_filter) = &mut self.key_filter else {
|
||||
return Ok(());
|
||||
};
|
||||
|
||||
while self.parts_reader.is_valid() {
|
||||
let pk_index = self.parts_reader.current_data_batch().pk_index();
|
||||
if let Some(yield_pk_index) = self.last_yield_pk_index
|
||||
&& pk_index == yield_pk_index
|
||||
{
|
||||
break;
|
||||
}
|
||||
self.keys_before_pruning += 1;
|
||||
// Safety: `key_filter` is some so the shard has primary keys.
|
||||
let key = self.key_dict.as_ref().unwrap().key_by_pk_index(pk_index);
|
||||
let now = Instant::now();
|
||||
if key_filter.matches(key).context(DecodeSnafu)? {
|
||||
self.prune_pk_cost += now.elapsed();
|
||||
self.last_yield_pk_index = Some(pk_index);
|
||||
self.keys_after_pruning += 1;
|
||||
break;
|
||||
}
|
||||
self.prune_pk_cost += now.elapsed();
|
||||
self.parts_reader.next()?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for ShardReader {
|
||||
fn drop(&mut self) {
|
||||
let shard_prune_pk = self.prune_pk_cost.as_secs_f64();
|
||||
PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["shard_prune_pk"])
|
||||
.observe(shard_prune_pk);
|
||||
if self.keys_before_pruning > 0 {
|
||||
common_telemetry::debug!(
|
||||
"ShardReader metrics, data parts: {}, before pruning: {}, after pruning: {}, prune cost: {}s, build cost: {}s",
|
||||
self.parts_reader.num_parts(),
|
||||
self.keys_before_pruning,
|
||||
self.keys_after_pruning,
|
||||
shard_prune_pk,
|
||||
self.data_build_cost.as_secs_f64(),
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A merger that merges batches from multiple shards.
|
||||
pub(crate) struct ShardMerger {
|
||||
merger: Merger<ShardNode>,
|
||||
}
|
||||
|
||||
impl ShardMerger {
|
||||
pub(crate) fn try_new(nodes: Vec<ShardNode>) -> Result<Self> {
|
||||
let merger = Merger::try_new(nodes)?;
|
||||
Ok(ShardMerger { merger })
|
||||
}
|
||||
}
|
||||
|
||||
impl DataBatchSource for ShardMerger {
|
||||
fn is_valid(&self) -> bool {
|
||||
self.merger.is_valid()
|
||||
}
|
||||
|
||||
fn next(&mut self) -> Result<()> {
|
||||
self.merger.next()
|
||||
}
|
||||
|
||||
fn current_pk_id(&self) -> PkId {
|
||||
self.merger.current_node().current_pk_id()
|
||||
}
|
||||
|
||||
fn current_key(&self) -> Option<&[u8]> {
|
||||
self.merger.current_node().current_key()
|
||||
}
|
||||
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
let batch = self.merger.current_node().current_data_batch();
|
||||
batch.slice(0, self.merger.current_rows())
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) enum ShardSource {
|
||||
Builder(ShardBuilderReader),
|
||||
Shard(ShardReader),
|
||||
}
|
||||
|
||||
impl ShardSource {
|
||||
fn is_valid(&self) -> bool {
|
||||
match self {
|
||||
ShardSource::Builder(r) => r.is_valid(),
|
||||
ShardSource::Shard(r) => r.is_valid(),
|
||||
}
|
||||
}
|
||||
|
||||
fn next(&mut self) -> Result<()> {
|
||||
match self {
|
||||
ShardSource::Builder(r) => r.next(),
|
||||
ShardSource::Shard(r) => r.next(),
|
||||
}
|
||||
}
|
||||
|
||||
fn current_pk_id(&self) -> PkId {
|
||||
match self {
|
||||
ShardSource::Builder(r) => r.current_pk_id(),
|
||||
ShardSource::Shard(r) => r.current_pk_id(),
|
||||
}
|
||||
}
|
||||
|
||||
fn current_key(&self) -> Option<&[u8]> {
|
||||
match self {
|
||||
ShardSource::Builder(r) => r.current_key(),
|
||||
ShardSource::Shard(r) => r.current_key(),
|
||||
}
|
||||
}
|
||||
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
match self {
|
||||
ShardSource::Builder(r) => r.current_data_batch(),
|
||||
ShardSource::Shard(r) => r.current_data_batch(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Node for the merger to get items.
|
||||
pub(crate) struct ShardNode {
|
||||
source: ShardSource,
|
||||
}
|
||||
|
||||
impl ShardNode {
|
||||
pub(crate) fn new(source: ShardSource) -> Self {
|
||||
Self { source }
|
||||
}
|
||||
|
||||
fn current_pk_id(&self) -> PkId {
|
||||
self.source.current_pk_id()
|
||||
}
|
||||
|
||||
fn current_key(&self) -> Option<&[u8]> {
|
||||
self.source.current_key()
|
||||
}
|
||||
|
||||
fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
self.source.current_data_batch()
|
||||
}
|
||||
}
|
||||
|
||||
impl PartialEq for ShardNode {
|
||||
fn eq(&self, other: &Self) -> bool {
|
||||
self.source.current_key() == other.source.current_key()
|
||||
}
|
||||
}
|
||||
|
||||
impl Eq for ShardNode {}
|
||||
|
||||
impl Ord for ShardNode {
|
||||
fn cmp(&self, other: &Self) -> Ordering {
|
||||
self.source
|
||||
.current_key()
|
||||
.cmp(&other.source.current_key())
|
||||
.reverse()
|
||||
}
|
||||
}
|
||||
|
||||
impl PartialOrd for ShardNode {
|
||||
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
|
||||
Some(self.cmp(other))
|
||||
}
|
||||
}
|
||||
|
||||
impl Node for ShardNode {
|
||||
fn is_valid(&self) -> bool {
|
||||
self.source.is_valid()
|
||||
}
|
||||
|
||||
fn is_behind(&self, other: &Self) -> bool {
|
||||
// We expect a key only belongs to one shard.
|
||||
debug_assert_ne!(self.source.current_key(), other.source.current_key());
|
||||
self.source.current_key() < other.source.current_key()
|
||||
}
|
||||
|
||||
fn advance(&mut self, len: usize) -> Result<()> {
|
||||
debug_assert_eq!(self.source.current_data_batch().num_rows(), len);
|
||||
self.source.next()
|
||||
}
|
||||
|
||||
fn current_item_len(&self) -> usize {
|
||||
self.current_data_batch().num_rows()
|
||||
}
|
||||
|
||||
fn search_key_in_current_item(&self, _other: &Self) -> Result<usize, usize> {
|
||||
Err(self.source.current_data_batch().num_rows())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use super::*;
|
||||
use crate::memtable::KeyValues;
|
||||
use crate::memtable::partition_tree::PkIndex;
|
||||
use crate::memtable::partition_tree::data::timestamp_array_to_i64_slice;
|
||||
use crate::memtable::partition_tree::dict::KeyDictBuilder;
|
||||
use crate::memtable::stats::WriteMetrics;
|
||||
use crate::test_util::memtable_util::{
|
||||
build_key_values_with_ts_seq_values, encode_keys, metadata_for_test,
|
||||
};
|
||||
|
||||
/// Returns key values and expect pk index.
|
||||
fn input_with_key(metadata: &RegionMetadataRef) -> Vec<(KeyValues, PkIndex)> {
|
||||
vec![
|
||||
(
|
||||
build_key_values_with_ts_seq_values(
|
||||
metadata,
|
||||
"shard".to_string(),
|
||||
2,
|
||||
[20, 21].into_iter(),
|
||||
[Some(0.0), Some(1.0)].into_iter(),
|
||||
0,
|
||||
),
|
||||
2,
|
||||
),
|
||||
(
|
||||
build_key_values_with_ts_seq_values(
|
||||
metadata,
|
||||
"shard".to_string(),
|
||||
0,
|
||||
[0, 1].into_iter(),
|
||||
[Some(0.0), Some(1.0)].into_iter(),
|
||||
1,
|
||||
),
|
||||
0,
|
||||
),
|
||||
(
|
||||
build_key_values_with_ts_seq_values(
|
||||
metadata,
|
||||
"shard".to_string(),
|
||||
1,
|
||||
[10, 11].into_iter(),
|
||||
[Some(0.0), Some(1.0)].into_iter(),
|
||||
2,
|
||||
),
|
||||
1,
|
||||
),
|
||||
]
|
||||
}
|
||||
|
||||
fn new_shard_with_dict(
|
||||
shard_id: ShardId,
|
||||
metadata: RegionMetadataRef,
|
||||
input: &[(KeyValues, PkIndex)],
|
||||
data_freeze_threshold: usize,
|
||||
) -> Shard {
|
||||
let mut dict_builder = KeyDictBuilder::new(1024);
|
||||
let mut metrics = WriteMetrics::default();
|
||||
let mut keys = Vec::with_capacity(input.len());
|
||||
for (kvs, _) in input {
|
||||
encode_keys(&metadata, kvs, &mut keys);
|
||||
}
|
||||
for key in &keys {
|
||||
dict_builder.insert_key(key, None, &mut metrics);
|
||||
}
|
||||
|
||||
let (dict, _) = dict_builder.finish().unwrap();
|
||||
let data_parts = DataParts::new(metadata, DATA_INIT_CAP, true);
|
||||
|
||||
Shard::new(
|
||||
shard_id,
|
||||
Some(Arc::new(dict)),
|
||||
data_parts,
|
||||
true,
|
||||
data_freeze_threshold,
|
||||
)
|
||||
}
|
||||
|
||||
fn collect_timestamps(shard: &Shard) -> Vec<i64> {
|
||||
let mut reader = shard.read().unwrap().build(None).unwrap();
|
||||
let mut timestamps = Vec::new();
|
||||
while reader.is_valid() {
|
||||
let rb = reader.current_data_batch().slice_record_batch();
|
||||
let ts_array = rb.column(1);
|
||||
let ts_slice = timestamp_array_to_i64_slice(ts_array);
|
||||
timestamps.extend_from_slice(ts_slice);
|
||||
|
||||
reader.next().unwrap();
|
||||
}
|
||||
timestamps
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_read_shard() {
|
||||
let metadata = metadata_for_test();
|
||||
let input = input_with_key(&metadata);
|
||||
let mut shard = new_shard_with_dict(8, metadata, &input, 100);
|
||||
assert!(shard.is_empty());
|
||||
for (key_values, pk_index) in &input {
|
||||
for kv in key_values.iter() {
|
||||
let pk_id = PkId {
|
||||
shard_id: shard.shard_id,
|
||||
pk_index: *pk_index,
|
||||
};
|
||||
shard.write_with_pk_id(pk_id, &kv).unwrap();
|
||||
}
|
||||
}
|
||||
assert!(!shard.is_empty());
|
||||
|
||||
let timestamps = collect_timestamps(&shard);
|
||||
assert_eq!(vec![0, 1, 10, 11, 20, 21], timestamps);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_shard_freeze() {
|
||||
let metadata = metadata_for_test();
|
||||
let kvs = build_key_values_with_ts_seq_values(
|
||||
&metadata,
|
||||
"shard".to_string(),
|
||||
0,
|
||||
[0].into_iter(),
|
||||
[Some(0.0)].into_iter(),
|
||||
0,
|
||||
);
|
||||
let mut shard = new_shard_with_dict(8, metadata.clone(), &[(kvs, 0)], 50);
|
||||
let expected: Vec<_> = (0..200).collect();
|
||||
for i in &expected {
|
||||
let kvs = build_key_values_with_ts_seq_values(
|
||||
&metadata,
|
||||
"shard".to_string(),
|
||||
0,
|
||||
[*i].into_iter(),
|
||||
[Some(0.0)].into_iter(),
|
||||
*i as u64,
|
||||
);
|
||||
let pk_id = PkId {
|
||||
shard_id: shard.shard_id,
|
||||
pk_index: *i as PkIndex,
|
||||
};
|
||||
for kv in kvs.iter() {
|
||||
shard.write_with_pk_id(pk_id, &kv).unwrap();
|
||||
}
|
||||
}
|
||||
assert!(!shard.is_empty());
|
||||
assert_eq!(3, shard.data_parts.frozen_len());
|
||||
|
||||
let timestamps = collect_timestamps(&shard);
|
||||
assert_eq!(expected, timestamps);
|
||||
}
|
||||
}
|
||||
@@ -1,418 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//! Builder of a shard.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use mito_codec::key_values::KeyValue;
|
||||
use mito_codec::row_converter::PrimaryKeyFilter;
|
||||
use snafu::ResultExt;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
|
||||
use crate::error::{DecodeSnafu, Result};
|
||||
use crate::memtable::partition_tree::data::{
|
||||
DATA_INIT_CAP, DataBatch, DataBuffer, DataBufferReader, DataBufferReaderBuilder, DataParts,
|
||||
};
|
||||
use crate::memtable::partition_tree::dict::{DictBuilderReader, KeyDictBuilder};
|
||||
use crate::memtable::partition_tree::shard::Shard;
|
||||
use crate::memtable::partition_tree::{PartitionTreeConfig, PkId, PkIndex, ShardId};
|
||||
use crate::memtable::stats::WriteMetrics;
|
||||
use crate::metrics::PARTITION_TREE_READ_STAGE_ELAPSED;
|
||||
|
||||
/// Builder to write keys and data to a shard that the key dictionary
|
||||
/// is still active.
|
||||
pub struct ShardBuilder {
|
||||
/// Id of the current shard to build.
|
||||
current_shard_id: ShardId,
|
||||
/// Builder for the key dictionary.
|
||||
dict_builder: KeyDictBuilder,
|
||||
/// Buffer to store data.
|
||||
data_buffer: DataBuffer,
|
||||
/// Number of rows to freeze a data part.
|
||||
data_freeze_threshold: usize,
|
||||
dedup: bool,
|
||||
}
|
||||
|
||||
impl ShardBuilder {
|
||||
/// Returns a new builder.
|
||||
pub fn new(
|
||||
metadata: RegionMetadataRef,
|
||||
config: &PartitionTreeConfig,
|
||||
shard_id: ShardId,
|
||||
) -> ShardBuilder {
|
||||
ShardBuilder {
|
||||
current_shard_id: shard_id,
|
||||
dict_builder: KeyDictBuilder::new(config.index_max_keys_per_shard),
|
||||
data_buffer: DataBuffer::with_capacity(metadata, DATA_INIT_CAP, config.dedup),
|
||||
data_freeze_threshold: config.data_freeze_threshold,
|
||||
dedup: config.dedup,
|
||||
}
|
||||
}
|
||||
|
||||
/// Write a key value with given pk_index (caller must ensure the pk_index exist in dict_builder)
|
||||
pub fn write_with_pk_id(&mut self, pk_id: PkId, key_value: &KeyValue) {
|
||||
assert_eq!(self.current_shard_id, pk_id.shard_id);
|
||||
self.data_buffer.write_row(pk_id.pk_index, key_value);
|
||||
}
|
||||
|
||||
/// Write a key value with its encoded primary key.
|
||||
pub fn write_with_key(
|
||||
&mut self,
|
||||
full_primary_key: &[u8],
|
||||
sparse_key: Option<&[u8]>,
|
||||
key_value: &KeyValue,
|
||||
metrics: &mut WriteMetrics,
|
||||
) -> PkId {
|
||||
// Safety: we check whether the builder need to freeze before.
|
||||
let pk_index = self
|
||||
.dict_builder
|
||||
.insert_key(full_primary_key, sparse_key, metrics);
|
||||
self.data_buffer.write_row(pk_index, key_value);
|
||||
PkId {
|
||||
shard_id: self.current_shard_id,
|
||||
pk_index,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the builder need to freeze.
|
||||
pub fn should_freeze(&self) -> bool {
|
||||
self.dict_builder.is_full() || self.data_buffer.num_rows() == self.data_freeze_threshold
|
||||
}
|
||||
|
||||
/// Returns the current shard id of the builder.
|
||||
pub fn current_shard_id(&self) -> ShardId {
|
||||
self.current_shard_id
|
||||
}
|
||||
|
||||
/// Builds a new shard and resets the builder.
|
||||
///
|
||||
/// Returns `None` if the builder is empty.
|
||||
pub fn finish(
|
||||
&mut self,
|
||||
metadata: RegionMetadataRef,
|
||||
pk_to_pk_id: &mut HashMap<Vec<u8>, PkId>,
|
||||
) -> Result<Option<Shard>> {
|
||||
if self.is_empty() {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let (data_part, key_dict) = match self.dict_builder.finish() {
|
||||
Some((dict, pk_to_index)) => {
|
||||
// Adds mapping to the map.
|
||||
pk_to_pk_id.reserve(pk_to_index.len());
|
||||
for (k, pk_index) in pk_to_index {
|
||||
pk_to_pk_id.insert(
|
||||
k,
|
||||
PkId {
|
||||
shard_id: self.current_shard_id,
|
||||
pk_index,
|
||||
},
|
||||
);
|
||||
}
|
||||
|
||||
let pk_weights = dict.pk_weights_to_sort_data();
|
||||
let part = self.data_buffer.freeze(Some(&pk_weights), true)?;
|
||||
(part, Some(dict))
|
||||
}
|
||||
None => {
|
||||
let pk_weights = [0];
|
||||
(self.data_buffer.freeze(Some(&pk_weights), true)?, None)
|
||||
}
|
||||
};
|
||||
|
||||
// build data parts.
|
||||
let data_parts =
|
||||
DataParts::new(metadata, DATA_INIT_CAP, self.dedup).with_frozen(vec![data_part]);
|
||||
let key_dict = key_dict.map(Arc::new);
|
||||
let shard_id = self.current_shard_id;
|
||||
self.current_shard_id += 1;
|
||||
|
||||
Ok(Some(Shard::new(
|
||||
shard_id,
|
||||
key_dict,
|
||||
data_parts,
|
||||
self.dedup,
|
||||
self.data_freeze_threshold,
|
||||
)))
|
||||
}
|
||||
|
||||
/// Scans the shard builder.
|
||||
pub fn read(&self, pk_weights_buffer: &mut Vec<u16>) -> Result<ShardBuilderReaderBuilder> {
|
||||
let dict_reader = {
|
||||
let _timer = PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["shard_builder_read_pk"])
|
||||
.start_timer();
|
||||
self.dict_builder.read()
|
||||
};
|
||||
|
||||
{
|
||||
let _timer = PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["sort_pk"])
|
||||
.start_timer();
|
||||
dict_reader.pk_weights_to_sort_data(pk_weights_buffer);
|
||||
}
|
||||
|
||||
let data_reader = self.data_buffer.read()?;
|
||||
Ok(ShardBuilderReaderBuilder {
|
||||
shard_id: self.current_shard_id,
|
||||
dict_reader,
|
||||
data_reader,
|
||||
})
|
||||
}
|
||||
|
||||
/// Returns true if the builder is empty.
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.data_buffer.is_empty()
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct ShardBuilderReaderBuilder {
|
||||
shard_id: ShardId,
|
||||
dict_reader: DictBuilderReader,
|
||||
data_reader: DataBufferReaderBuilder,
|
||||
}
|
||||
|
||||
impl ShardBuilderReaderBuilder {
|
||||
pub(crate) fn build(
|
||||
self,
|
||||
pk_weights: Option<&[u16]>,
|
||||
key_filter: Option<Box<dyn PrimaryKeyFilter>>,
|
||||
) -> Result<ShardBuilderReader> {
|
||||
let now = Instant::now();
|
||||
let data_reader = self.data_reader.build(pk_weights)?;
|
||||
ShardBuilderReader::new(
|
||||
self.shard_id,
|
||||
self.dict_reader,
|
||||
data_reader,
|
||||
key_filter,
|
||||
now.elapsed(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/// Reader to scan a shard builder.
|
||||
pub struct ShardBuilderReader {
|
||||
shard_id: ShardId,
|
||||
dict_reader: DictBuilderReader,
|
||||
data_reader: DataBufferReader,
|
||||
key_filter: Option<Box<dyn PrimaryKeyFilter>>,
|
||||
last_yield_pk_index: Option<PkIndex>,
|
||||
keys_before_pruning: usize,
|
||||
keys_after_pruning: usize,
|
||||
prune_pk_cost: Duration,
|
||||
data_build_cost: Duration,
|
||||
}
|
||||
|
||||
impl ShardBuilderReader {
|
||||
fn new(
|
||||
shard_id: ShardId,
|
||||
dict_reader: DictBuilderReader,
|
||||
data_reader: DataBufferReader,
|
||||
key_filter: Option<Box<dyn PrimaryKeyFilter>>,
|
||||
data_build_cost: Duration,
|
||||
) -> Result<Self> {
|
||||
let mut reader = ShardBuilderReader {
|
||||
shard_id,
|
||||
dict_reader,
|
||||
data_reader,
|
||||
key_filter,
|
||||
last_yield_pk_index: None,
|
||||
keys_before_pruning: 0,
|
||||
keys_after_pruning: 0,
|
||||
prune_pk_cost: Duration::default(),
|
||||
data_build_cost,
|
||||
};
|
||||
reader.prune_batch_by_key()?;
|
||||
|
||||
Ok(reader)
|
||||
}
|
||||
|
||||
pub fn is_valid(&self) -> bool {
|
||||
self.data_reader.is_valid()
|
||||
}
|
||||
|
||||
pub fn next(&mut self) -> Result<()> {
|
||||
self.data_reader.next()?;
|
||||
self.prune_batch_by_key()
|
||||
}
|
||||
|
||||
pub fn current_key(&self) -> Option<&[u8]> {
|
||||
let pk_index = self.data_reader.current_data_batch().pk_index();
|
||||
Some(self.dict_reader.key_by_pk_index(pk_index))
|
||||
}
|
||||
|
||||
pub fn current_pk_id(&self) -> PkId {
|
||||
let pk_index = self.data_reader.current_data_batch().pk_index();
|
||||
PkId {
|
||||
shard_id: self.shard_id,
|
||||
pk_index,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn current_data_batch(&self) -> DataBatch<'_> {
|
||||
self.data_reader.current_data_batch()
|
||||
}
|
||||
|
||||
fn prune_batch_by_key(&mut self) -> Result<()> {
|
||||
let Some(key_filter) = &mut self.key_filter else {
|
||||
return Ok(());
|
||||
};
|
||||
|
||||
while self.data_reader.is_valid() {
|
||||
let pk_index = self.data_reader.current_data_batch().pk_index();
|
||||
if let Some(yield_pk_index) = self.last_yield_pk_index
|
||||
&& pk_index == yield_pk_index
|
||||
{
|
||||
break;
|
||||
}
|
||||
self.keys_before_pruning += 1;
|
||||
let key = self.dict_reader.key_by_pk_index(pk_index);
|
||||
let now = Instant::now();
|
||||
if key_filter.matches(key).context(DecodeSnafu)? {
|
||||
self.prune_pk_cost += now.elapsed();
|
||||
self.last_yield_pk_index = Some(pk_index);
|
||||
self.keys_after_pruning += 1;
|
||||
break;
|
||||
}
|
||||
self.prune_pk_cost += now.elapsed();
|
||||
self.data_reader.next()?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for ShardBuilderReader {
|
||||
fn drop(&mut self) {
|
||||
let shard_builder_prune_pk = self.prune_pk_cost.as_secs_f64();
|
||||
PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["shard_builder_prune_pk"])
|
||||
.observe(shard_builder_prune_pk);
|
||||
if self.keys_before_pruning > 0 {
|
||||
common_telemetry::debug!(
|
||||
"ShardBuilderReader metrics, before pruning: {}, after pruning: {}, prune cost: {}s, build cost: {}s",
|
||||
self.keys_before_pruning,
|
||||
self.keys_after_pruning,
|
||||
shard_builder_prune_pk,
|
||||
self.data_build_cost.as_secs_f64(),
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::memtable::KeyValues;
|
||||
use crate::memtable::partition_tree::data::timestamp_array_to_i64_slice;
|
||||
use crate::test_util::memtable_util::{
|
||||
build_key_values_with_ts_seq_values, encode_key_by_kv, metadata_for_test,
|
||||
};
|
||||
|
||||
fn input_with_key(metadata: &RegionMetadataRef) -> Vec<KeyValues> {
|
||||
vec![
|
||||
build_key_values_with_ts_seq_values(
|
||||
metadata,
|
||||
"shard_builder".to_string(),
|
||||
2,
|
||||
[20, 21].into_iter(),
|
||||
[Some(0.0), Some(1.0)].into_iter(),
|
||||
0,
|
||||
),
|
||||
build_key_values_with_ts_seq_values(
|
||||
metadata,
|
||||
"shard_builder".to_string(),
|
||||
0,
|
||||
[0, 1].into_iter(),
|
||||
[Some(0.0), Some(1.0)].into_iter(),
|
||||
1,
|
||||
),
|
||||
build_key_values_with_ts_seq_values(
|
||||
metadata,
|
||||
"shard_builder".to_string(),
|
||||
1,
|
||||
[10, 11].into_iter(),
|
||||
[Some(0.0), Some(1.0)].into_iter(),
|
||||
2,
|
||||
),
|
||||
]
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_shard_builder() {
|
||||
let metadata = metadata_for_test();
|
||||
let input = input_with_key(&metadata);
|
||||
let config = PartitionTreeConfig::default();
|
||||
let mut shard_builder = ShardBuilder::new(metadata.clone(), &config, 1);
|
||||
let mut metrics = WriteMetrics::default();
|
||||
assert!(
|
||||
shard_builder
|
||||
.finish(metadata.clone(), &mut HashMap::new())
|
||||
.unwrap()
|
||||
.is_none()
|
||||
);
|
||||
assert_eq!(1, shard_builder.current_shard_id);
|
||||
|
||||
for key_values in &input {
|
||||
for kv in key_values.iter() {
|
||||
let key = encode_key_by_kv(&kv);
|
||||
shard_builder.write_with_key(&key, None, &kv, &mut metrics);
|
||||
}
|
||||
}
|
||||
let shard = shard_builder
|
||||
.finish(metadata, &mut HashMap::new())
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
assert_eq!(1, shard.shard_id);
|
||||
assert_eq!(2, shard_builder.current_shard_id);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_write_read_shard_builder() {
|
||||
let metadata = metadata_for_test();
|
||||
let input = input_with_key(&metadata);
|
||||
let config = PartitionTreeConfig::default();
|
||||
let mut shard_builder = ShardBuilder::new(metadata.clone(), &config, 1);
|
||||
let mut metrics = WriteMetrics::default();
|
||||
|
||||
for key_values in &input {
|
||||
for kv in key_values.iter() {
|
||||
let key = encode_key_by_kv(&kv);
|
||||
shard_builder.write_with_key(&key, None, &kv, &mut metrics);
|
||||
}
|
||||
}
|
||||
|
||||
let mut pk_weights = Vec::new();
|
||||
let mut reader = shard_builder
|
||||
.read(&mut pk_weights)
|
||||
.unwrap()
|
||||
.build(Some(&pk_weights), None)
|
||||
.unwrap();
|
||||
let mut timestamps = Vec::new();
|
||||
while reader.is_valid() {
|
||||
let rb = reader.current_data_batch().slice_record_batch();
|
||||
let ts_array = rb.column(1);
|
||||
let ts_slice = timestamp_array_to_i64_slice(ts_array);
|
||||
timestamps.extend_from_slice(ts_slice);
|
||||
|
||||
reader.next().unwrap();
|
||||
}
|
||||
assert_eq!(vec![0, 1, 10, 11, 20, 21], timestamps);
|
||||
assert_eq!(vec![2, 0, 1], pk_weights);
|
||||
}
|
||||
}
|
||||
@@ -1,584 +0,0 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
//! Implementation of the partition tree.
|
||||
|
||||
use std::collections::{BTreeMap, HashSet, VecDeque};
|
||||
use std::sync::{Arc, RwLock};
|
||||
use std::time::{Duration, Instant};
|
||||
|
||||
use api::v1::OpType;
|
||||
use common_recordbatch::filter::SimpleFilterEvaluator;
|
||||
use common_time::Timestamp;
|
||||
use datafusion_common::ScalarValue;
|
||||
use datatypes::prelude::ValueRef;
|
||||
use mito_codec::key_values::KeyValue;
|
||||
use mito_codec::primary_key_filter::is_partition_column;
|
||||
use mito_codec::row_converter::sparse::{FieldWithId, SparseEncoder};
|
||||
use mito_codec::row_converter::{PrimaryKeyCodec, SortField};
|
||||
use snafu::{ResultExt, ensure};
|
||||
use store_api::codec::PrimaryKeyEncoding;
|
||||
use store_api::metadata::RegionMetadataRef;
|
||||
use store_api::storage::{ColumnId, SequenceRange};
|
||||
use table::predicate::Predicate;
|
||||
|
||||
use crate::error::{
|
||||
EncodeSnafu, EncodeSparsePrimaryKeySnafu, PrimaryKeyLengthMismatchSnafu, Result,
|
||||
};
|
||||
use crate::flush::WriteBufferManagerRef;
|
||||
use crate::memtable::partition_tree::PartitionTreeConfig;
|
||||
use crate::memtable::partition_tree::partition::{
|
||||
Partition, PartitionKey, PartitionReader, PartitionRef, ReadPartitionContext,
|
||||
};
|
||||
use crate::memtable::stats::WriteMetrics;
|
||||
use crate::memtable::{BoxedBatchIterator, KeyValues};
|
||||
use crate::metrics::{PARTITION_TREE_READ_STAGE_ELAPSED, READ_ROWS_TOTAL, READ_STAGE_ELAPSED};
|
||||
use crate::read::Batch;
|
||||
use crate::read::dedup::LastNonNullIter;
|
||||
use crate::region::options::MergeMode;
|
||||
|
||||
/// The partition tree.
|
||||
pub struct PartitionTree {
|
||||
/// Config of the tree.
|
||||
config: PartitionTreeConfig,
|
||||
/// Metadata of the region.
|
||||
pub(crate) metadata: RegionMetadataRef,
|
||||
/// Primary key codec.
|
||||
row_codec: Arc<dyn PrimaryKeyCodec>,
|
||||
/// Partitions in the tree.
|
||||
partitions: RwLock<BTreeMap<PartitionKey, PartitionRef>>,
|
||||
/// Whether the tree has multiple partitions.
|
||||
is_partitioned: bool,
|
||||
/// Manager to report size of the tree.
|
||||
write_buffer_manager: Option<WriteBufferManagerRef>,
|
||||
sparse_encoder: Arc<SparseEncoder>,
|
||||
}
|
||||
|
||||
impl PartitionTree {
|
||||
/// Creates a new partition tree.
|
||||
pub fn new(
|
||||
row_codec: Arc<dyn PrimaryKeyCodec>,
|
||||
metadata: RegionMetadataRef,
|
||||
config: &PartitionTreeConfig,
|
||||
write_buffer_manager: Option<WriteBufferManagerRef>,
|
||||
) -> Self {
|
||||
let sparse_encoder = SparseEncoder::new(
|
||||
metadata
|
||||
.primary_key_columns()
|
||||
.map(|c| FieldWithId {
|
||||
field: SortField::new(c.column_schema.data_type.clone()),
|
||||
column_id: c.column_id,
|
||||
})
|
||||
.collect(),
|
||||
);
|
||||
let is_partitioned = Partition::has_multi_partitions(&metadata);
|
||||
let mut config = config.clone();
|
||||
if config.merge_mode == MergeMode::LastNonNull {
|
||||
config.dedup = false;
|
||||
}
|
||||
|
||||
PartitionTree {
|
||||
config,
|
||||
metadata,
|
||||
row_codec,
|
||||
partitions: Default::default(),
|
||||
is_partitioned,
|
||||
write_buffer_manager,
|
||||
sparse_encoder: Arc::new(sparse_encoder),
|
||||
}
|
||||
}
|
||||
|
||||
fn verify_primary_key_length(&self, kv: &KeyValue) -> Result<()> {
|
||||
// The sparse primary key codec does not have a fixed number of fields.
|
||||
if let Some(expected_num_fields) = self.row_codec.num_fields() {
|
||||
ensure!(
|
||||
expected_num_fields == kv.num_primary_keys(),
|
||||
PrimaryKeyLengthMismatchSnafu {
|
||||
expect: expected_num_fields,
|
||||
actual: kv.num_primary_keys(),
|
||||
}
|
||||
);
|
||||
}
|
||||
// TODO(weny): verify the primary key length for sparse primary key codec.
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Encodes the given key value into a sparse primary key.
|
||||
fn encode_sparse_primary_key(&self, kv: &KeyValue, buffer: &mut Vec<u8>) -> Result<()> {
|
||||
if kv.primary_key_encoding() == PrimaryKeyEncoding::Sparse {
|
||||
// If the primary key encoding is sparse and already encoded in the metric engine,
|
||||
// we only need to copy the encoded primary key into the destination buffer.
|
||||
let ValueRef::Binary(primary_key) = kv.primary_keys().next().unwrap() else {
|
||||
return EncodeSparsePrimaryKeySnafu {
|
||||
reason: "sparse primary key is not binary".to_string(),
|
||||
}
|
||||
.fail();
|
||||
};
|
||||
buffer.extend_from_slice(primary_key);
|
||||
} else {
|
||||
// For compatibility, use the sparse encoder for dense primary key.
|
||||
self.sparse_encoder
|
||||
.encode_to_vec(kv.primary_keys(), buffer)
|
||||
.context(EncodeSnafu)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// TODO(yingwen): The size computed from values is inaccurate.
|
||||
/// Write key-values into the tree.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the tree is immutable (frozen).
|
||||
pub fn write(
|
||||
&self,
|
||||
kvs: &KeyValues,
|
||||
pk_buffer: &mut Vec<u8>,
|
||||
metrics: &mut WriteMetrics,
|
||||
) -> Result<()> {
|
||||
let has_pk = !self.metadata.primary_key.is_empty();
|
||||
|
||||
for kv in kvs.iter() {
|
||||
self.verify_primary_key_length(&kv)?;
|
||||
// Safety: timestamp of kv must be both present and a valid timestamp value.
|
||||
let ts = kv
|
||||
.timestamp()
|
||||
.try_into_timestamp()
|
||||
.unwrap()
|
||||
.unwrap()
|
||||
.value();
|
||||
metrics.min_ts = metrics.min_ts.min(ts);
|
||||
metrics.max_ts = metrics.max_ts.max(ts);
|
||||
metrics.value_bytes += kv.fields().map(|v| v.data_size()).sum::<usize>();
|
||||
|
||||
if !has_pk {
|
||||
// No primary key.
|
||||
self.write_no_key(kv)?;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Encode primary key.
|
||||
pk_buffer.clear();
|
||||
if self.is_partitioned {
|
||||
self.encode_sparse_primary_key(&kv, pk_buffer)?;
|
||||
} else {
|
||||
self.row_codec
|
||||
.encode_key_value(&kv, pk_buffer)
|
||||
.context(EncodeSnafu)?;
|
||||
}
|
||||
|
||||
// Write rows with
|
||||
self.write_with_key(pk_buffer, kv, metrics)?;
|
||||
}
|
||||
|
||||
metrics.value_bytes +=
|
||||
kvs.num_rows() * (std::mem::size_of::<Timestamp>() + std::mem::size_of::<OpType>());
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Write one key value pair into the tree.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if the tree is immutable (frozen).
|
||||
pub fn write_one(
|
||||
&self,
|
||||
kv: KeyValue,
|
||||
pk_buffer: &mut Vec<u8>,
|
||||
metrics: &mut WriteMetrics,
|
||||
) -> Result<()> {
|
||||
let has_pk = !self.metadata.primary_key.is_empty();
|
||||
|
||||
self.verify_primary_key_length(&kv)?;
|
||||
// Safety: timestamp of kv must be both present and a valid timestamp value.
|
||||
let ts = kv
|
||||
.timestamp()
|
||||
.try_into_timestamp()
|
||||
.unwrap()
|
||||
.unwrap()
|
||||
.value();
|
||||
metrics.min_ts = metrics.min_ts.min(ts);
|
||||
metrics.max_ts = metrics.max_ts.max(ts);
|
||||
metrics.value_bytes += kv.fields().map(|v| v.data_size()).sum::<usize>();
|
||||
|
||||
if !has_pk {
|
||||
// No primary key.
|
||||
return self.write_no_key(kv);
|
||||
}
|
||||
|
||||
// Encode primary key.
|
||||
pk_buffer.clear();
|
||||
if self.is_partitioned {
|
||||
self.encode_sparse_primary_key(&kv, pk_buffer)?;
|
||||
} else {
|
||||
self.row_codec
|
||||
.encode_key_value(&kv, pk_buffer)
|
||||
.context(EncodeSnafu)?;
|
||||
}
|
||||
|
||||
// Write rows with
|
||||
self.write_with_key(pk_buffer, kv, metrics)?;
|
||||
|
||||
metrics.value_bytes += std::mem::size_of::<Timestamp>() + std::mem::size_of::<OpType>();
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Scans the tree.
|
||||
pub fn read(
|
||||
&self,
|
||||
projection: Option<&[ColumnId]>,
|
||||
predicate: Option<Predicate>,
|
||||
sequence: Option<SequenceRange>,
|
||||
mem_scan_metrics: Option<crate::memtable::MemScanMetrics>,
|
||||
) -> Result<BoxedBatchIterator> {
|
||||
let start = Instant::now();
|
||||
// Creates the projection set.
|
||||
let projection: HashSet<_> = if let Some(projection) = projection {
|
||||
projection.iter().copied().collect()
|
||||
} else {
|
||||
self.metadata.field_columns().map(|c| c.column_id).collect()
|
||||
};
|
||||
|
||||
let filters = predicate
|
||||
.map(|predicate| {
|
||||
predicate
|
||||
.exprs()
|
||||
.iter()
|
||||
.filter_map(SimpleFilterEvaluator::try_new)
|
||||
.collect::<Vec<_>>()
|
||||
})
|
||||
.unwrap_or_default();
|
||||
|
||||
let mut tree_iter_metric = TreeIterMetrics::default();
|
||||
let partitions = self.prune_partitions(&filters, &mut tree_iter_metric);
|
||||
|
||||
let mut iter = TreeIter {
|
||||
sequence,
|
||||
partitions,
|
||||
current_reader: None,
|
||||
metrics: tree_iter_metric,
|
||||
mem_scan_metrics,
|
||||
};
|
||||
let context = ReadPartitionContext::new(
|
||||
self.metadata.clone(),
|
||||
self.row_codec.clone(),
|
||||
projection,
|
||||
Arc::new(filters),
|
||||
);
|
||||
iter.fetch_next_partition(context)?;
|
||||
|
||||
iter.metrics.iter_elapsed += start.elapsed();
|
||||
|
||||
if self.config.merge_mode == MergeMode::LastNonNull {
|
||||
let iter = LastNonNullIter::new(iter);
|
||||
Ok(Box::new(iter))
|
||||
} else {
|
||||
Ok(Box::new(iter))
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the tree is empty.
|
||||
///
|
||||
/// A tree is empty if no partition has data.
|
||||
pub fn is_empty(&self) -> bool {
|
||||
let partitions = self.partitions.read().unwrap();
|
||||
partitions.values().all(|part| !part.has_data())
|
||||
}
|
||||
|
||||
/// Marks the tree as immutable.
|
||||
///
|
||||
/// Once the tree becomes immutable, callers should not write to it again.
|
||||
pub fn freeze(&self) -> Result<()> {
|
||||
let partitions = self.partitions.read().unwrap();
|
||||
for partition in partitions.values() {
|
||||
partition.freeze()?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Forks an immutable tree. Returns a mutable tree that inherits the index
|
||||
/// of this tree.
|
||||
pub fn fork(&self, metadata: RegionMetadataRef) -> PartitionTree {
|
||||
if self.metadata.schema_version != metadata.schema_version
|
||||
|| self.metadata.column_metadatas != metadata.column_metadatas
|
||||
{
|
||||
// The schema has changed, we can't reuse the tree.
|
||||
return PartitionTree::new(
|
||||
self.row_codec.clone(),
|
||||
metadata,
|
||||
&self.config,
|
||||
self.write_buffer_manager.clone(),
|
||||
);
|
||||
}
|
||||
|
||||
let mut total_shared_size = 0;
|
||||
let mut part_infos = {
|
||||
let partitions = self.partitions.read().unwrap();
|
||||
partitions
|
||||
.iter()
|
||||
.filter_map(|(part_key, part)| {
|
||||
let stats = part.stats();
|
||||
if stats.num_rows > 0 {
|
||||
// Only fork partitions that have data.
|
||||
total_shared_size += stats.shared_memory_size;
|
||||
Some((*part_key, part.clone(), stats))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>()
|
||||
};
|
||||
|
||||
// TODO(yingwen): Optimize eviction strategy. Now we evict the whole partition.
|
||||
let fork_size = self.config.fork_dictionary_bytes.as_bytes() as usize;
|
||||
if total_shared_size > fork_size {
|
||||
// Sort partitions by memory size desc.
|
||||
part_infos.sort_unstable_by_key(|info| info.2.shared_memory_size);
|
||||
while total_shared_size > fork_size {
|
||||
let Some(info) = part_infos.pop() else {
|
||||
break;
|
||||
};
|
||||
|
||||
common_telemetry::debug!(
|
||||
"Evict partition {} with memory size {}, {} shards",
|
||||
info.0,
|
||||
info.2.shared_memory_size,
|
||||
info.2.shard_num,
|
||||
);
|
||||
|
||||
total_shared_size -= info.2.shared_memory_size;
|
||||
}
|
||||
}
|
||||
|
||||
let mut forked = BTreeMap::new();
|
||||
for (part_key, part, _) in part_infos {
|
||||
let forked_part = part.fork(&metadata, &self.config);
|
||||
forked.insert(part_key, Arc::new(forked_part));
|
||||
}
|
||||
|
||||
PartitionTree {
|
||||
config: self.config.clone(),
|
||||
metadata,
|
||||
row_codec: self.row_codec.clone(),
|
||||
partitions: RwLock::new(forked),
|
||||
is_partitioned: self.is_partitioned,
|
||||
write_buffer_manager: self.write_buffer_manager.clone(),
|
||||
sparse_encoder: self.sparse_encoder.clone(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the write buffer manager.
|
||||
pub(crate) fn write_buffer_manager(&self) -> Option<WriteBufferManagerRef> {
|
||||
self.write_buffer_manager.clone()
|
||||
}
|
||||
|
||||
fn write_with_key(
|
||||
&self,
|
||||
primary_key: &mut Vec<u8>,
|
||||
key_value: KeyValue,
|
||||
metrics: &mut WriteMetrics,
|
||||
) -> Result<()> {
|
||||
let partition_key = Partition::get_partition_key(&key_value, self.is_partitioned);
|
||||
let partition = self.get_or_create_partition(partition_key);
|
||||
|
||||
partition.write_with_key(
|
||||
primary_key,
|
||||
self.row_codec.as_ref(),
|
||||
key_value,
|
||||
self.is_partitioned, // If tree is partitioned, re-encode is required to get the full primary key.
|
||||
metrics,
|
||||
)
|
||||
}
|
||||
|
||||
fn write_no_key(&self, key_value: KeyValue) -> Result<()> {
|
||||
let partition_key = Partition::get_partition_key(&key_value, self.is_partitioned);
|
||||
let partition = self.get_or_create_partition(partition_key);
|
||||
|
||||
partition.write_no_key(key_value)
|
||||
}
|
||||
|
||||
fn get_or_create_partition(&self, partition_key: PartitionKey) -> PartitionRef {
|
||||
let mut partitions = self.partitions.write().unwrap();
|
||||
partitions
|
||||
.entry(partition_key)
|
||||
.or_insert_with(|| Arc::new(Partition::new(self.metadata.clone(), &self.config)))
|
||||
.clone()
|
||||
}
|
||||
|
||||
fn prune_partitions(
|
||||
&self,
|
||||
filters: &[SimpleFilterEvaluator],
|
||||
metrics: &mut TreeIterMetrics,
|
||||
) -> VecDeque<PartitionRef> {
|
||||
let partitions = self.partitions.read().unwrap();
|
||||
metrics.partitions_total = partitions.len();
|
||||
if !self.is_partitioned {
|
||||
return partitions.values().cloned().collect();
|
||||
}
|
||||
|
||||
let mut pruned = VecDeque::new();
|
||||
// Prune partition keys.
|
||||
for (key, partition) in partitions.iter() {
|
||||
let mut is_needed = true;
|
||||
for filter in filters {
|
||||
if !is_partition_column(filter.column_name()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if !filter
|
||||
.evaluate_scalar(&ScalarValue::UInt32(Some(*key)))
|
||||
.unwrap_or(true)
|
||||
{
|
||||
is_needed = false;
|
||||
}
|
||||
}
|
||||
|
||||
if is_needed {
|
||||
pruned.push_back(partition.clone());
|
||||
}
|
||||
}
|
||||
metrics.partitions_after_pruning = pruned.len();
|
||||
pruned
|
||||
}
|
||||
|
||||
/// Returns all series count in all partitions.
|
||||
pub(crate) fn series_count(&self) -> usize {
|
||||
self.partitions
|
||||
.read()
|
||||
.unwrap()
|
||||
.values()
|
||||
.map(|p| p.series_count())
|
||||
.sum()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
struct TreeIterMetrics {
|
||||
iter_elapsed: Duration,
|
||||
fetch_partition_elapsed: Duration,
|
||||
rows_fetched: usize,
|
||||
batches_fetched: usize,
|
||||
partitions_total: usize,
|
||||
partitions_after_pruning: usize,
|
||||
}
|
||||
|
||||
struct TreeIter {
|
||||
/// Optional Sequence number of the current reader which limit results batch to lower than this sequence number.
|
||||
sequence: Option<SequenceRange>,
|
||||
partitions: VecDeque<PartitionRef>,
|
||||
current_reader: Option<PartitionReader>,
|
||||
metrics: TreeIterMetrics,
|
||||
mem_scan_metrics: Option<crate::memtable::MemScanMetrics>,
|
||||
}
|
||||
|
||||
impl TreeIter {
|
||||
fn report_mem_scan_metrics(&mut self) {
|
||||
if let Some(mem_scan_metrics) = self.mem_scan_metrics.take() {
|
||||
let inner = crate::memtable::MemScanMetricsData {
|
||||
total_series: 0, // This is unavailable.
|
||||
num_rows: self.metrics.rows_fetched,
|
||||
num_batches: self.metrics.batches_fetched,
|
||||
scan_cost: self.metrics.iter_elapsed,
|
||||
..Default::default()
|
||||
};
|
||||
mem_scan_metrics.merge_inner(&inner);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for TreeIter {
|
||||
fn drop(&mut self) {
|
||||
// Report MemScanMetrics if not already reported
|
||||
self.report_mem_scan_metrics();
|
||||
|
||||
READ_ROWS_TOTAL
|
||||
.with_label_values(&["partition_tree_memtable"])
|
||||
.inc_by(self.metrics.rows_fetched as u64);
|
||||
PARTITION_TREE_READ_STAGE_ELAPSED
|
||||
.with_label_values(&["fetch_next_partition"])
|
||||
.observe(self.metrics.fetch_partition_elapsed.as_secs_f64());
|
||||
let scan_elapsed = self.metrics.iter_elapsed.as_secs_f64();
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["scan_memtable"])
|
||||
.observe(scan_elapsed);
|
||||
common_telemetry::debug!(
|
||||
"TreeIter partitions total: {}, partitions after prune: {}, rows fetched: {}, batches fetched: {}, scan elapsed: {}",
|
||||
self.metrics.partitions_total,
|
||||
self.metrics.partitions_after_pruning,
|
||||
self.metrics.rows_fetched,
|
||||
self.metrics.batches_fetched,
|
||||
scan_elapsed
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
impl Iterator for TreeIter {
|
||||
type Item = Result<Batch>;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
let start = Instant::now();
|
||||
let res = self.next_batch().transpose();
|
||||
self.metrics.iter_elapsed += start.elapsed();
|
||||
res
|
||||
}
|
||||
}
|
||||
|
||||
impl TreeIter {
|
||||
/// Fetch next partition.
|
||||
fn fetch_next_partition(&mut self, mut context: ReadPartitionContext) -> Result<()> {
|
||||
let start = Instant::now();
|
||||
while let Some(partition) = self.partitions.pop_front() {
|
||||
let part_reader = partition.read(context)?;
|
||||
if !part_reader.is_valid() {
|
||||
context = part_reader.into_context();
|
||||
continue;
|
||||
}
|
||||
self.current_reader = Some(part_reader);
|
||||
break;
|
||||
}
|
||||
self.metrics.fetch_partition_elapsed += start.elapsed();
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Fetches next batch.
|
||||
fn next_batch(&mut self) -> Result<Option<Batch>> {
|
||||
let Some(part_reader) = &mut self.current_reader else {
|
||||
// Report MemScanMetrics before returning None
|
||||
self.report_mem_scan_metrics();
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
debug_assert!(part_reader.is_valid());
|
||||
let batch = part_reader.convert_current_batch()?;
|
||||
part_reader.next()?;
|
||||
if part_reader.is_valid() {
|
||||
self.metrics.rows_fetched += batch.num_rows();
|
||||
self.metrics.batches_fetched += 1;
|
||||
let mut batch = batch;
|
||||
batch.filter_by_sequence(self.sequence)?;
|
||||
return Ok(Some(batch));
|
||||
}
|
||||
|
||||
// Safety: current reader is Some.
|
||||
let part_reader = self.current_reader.take().unwrap();
|
||||
let context = part_reader.into_context();
|
||||
self.fetch_next_partition(context)?;
|
||||
|
||||
self.metrics.rows_fetched += batch.num_rows();
|
||||
self.metrics.batches_fetched += 1;
|
||||
let mut batch = batch;
|
||||
batch.filter_by_sequence(self.sequence)?;
|
||||
Ok(Some(batch))
|
||||
}
|
||||
}
|
||||
@@ -825,7 +825,6 @@ mod tests {
|
||||
use store_api::storage::SequenceNumber;
|
||||
|
||||
use super::*;
|
||||
use crate::memtable::partition_tree::PartitionTreeMemtableBuilder;
|
||||
use crate::memtable::time_series::TimeSeriesMemtableBuilder;
|
||||
use crate::memtable::{IterBuilder, RangesOptions};
|
||||
use crate::test_util::memtable_util::{self, collect_iter_timestamps};
|
||||
@@ -833,7 +832,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_no_duration() {
|
||||
let metadata = memtable_util::metadata_for_test();
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
let partitions = TimePartitions::new(metadata.clone(), builder, 0, None);
|
||||
assert_eq!(0, partitions.num_partitions());
|
||||
assert!(partitions.is_empty());
|
||||
@@ -865,7 +864,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_write_single_part() {
|
||||
let metadata = memtable_util::metadata_for_test();
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
let partitions =
|
||||
TimePartitions::new(metadata.clone(), builder, 0, Some(Duration::from_secs(10)));
|
||||
assert_eq!(0, partitions.num_partitions());
|
||||
@@ -915,7 +914,7 @@ mod tests {
|
||||
|
||||
#[cfg(test)]
|
||||
fn new_multi_partitions(metadata: &RegionMetadataRef) -> TimePartitions {
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
let partitions =
|
||||
TimePartitions::new(metadata.clone(), builder, 0, Some(Duration::from_secs(5)));
|
||||
assert_eq!(0, partitions.num_partitions());
|
||||
@@ -991,7 +990,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_new_with_part_duration() {
|
||||
let metadata = memtable_util::metadata_for_test();
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
let partitions = TimePartitions::new(metadata.clone(), builder.clone(), 0, None);
|
||||
|
||||
let new_parts = partitions.new_with_part_duration(Some(Duration::from_secs(5)), None);
|
||||
@@ -1009,7 +1008,7 @@ mod tests {
|
||||
// Don't need to create new memtables.
|
||||
assert_eq!(0, new_parts.next_memtable_id());
|
||||
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
let partitions = TimePartitions::new(metadata.clone(), builder.clone(), 0, None);
|
||||
// Need to build a new memtable as duration is still None.
|
||||
let new_parts = partitions.new_with_part_duration(None, None);
|
||||
@@ -1020,7 +1019,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_fork_empty() {
|
||||
let metadata = memtable_util::metadata_for_test();
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
let partitions = TimePartitions::new(metadata.clone(), builder, 0, None);
|
||||
partitions.freeze().unwrap();
|
||||
let new_parts = partitions.fork(&metadata, None);
|
||||
@@ -1072,7 +1071,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_find_partitions_by_time_range() {
|
||||
let metadata = memtable_util::metadata_for_test();
|
||||
let builder = Arc::new(PartitionTreeMemtableBuilder::default());
|
||||
let builder = Arc::new(TimeSeriesMemtableBuilder::default());
|
||||
|
||||
// Case 1: No time range partitioning
|
||||
let partitions = TimePartitions::new(metadata.clone(), builder.clone(), 0, None);
|
||||
|
||||
@@ -417,29 +417,6 @@ lazy_static! {
|
||||
}
|
||||
|
||||
lazy_static! {
|
||||
/// Partition tree memtable data buffer freeze metrics
|
||||
pub static ref PARTITION_TREE_DATA_BUFFER_FREEZE_STAGE_ELAPSED: HistogramVec = register_histogram_vec!(
|
||||
"greptime_partition_tree_buffer_freeze_stage_elapsed",
|
||||
"mito partition tree data buffer freeze stage elapsed",
|
||||
&[STAGE_LABEL],
|
||||
// 0.01 ~ 1000
|
||||
exponential_buckets(0.01, 10.0, 6).unwrap(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
/// Partition tree memtable read path metrics
|
||||
pub static ref PARTITION_TREE_READ_STAGE_ELAPSED: HistogramVec = register_histogram_vec!(
|
||||
"greptime_partition_tree_read_stage_elapsed",
|
||||
"mito partition tree read stage elapsed",
|
||||
&[STAGE_LABEL],
|
||||
// 0.01 ~ 1000
|
||||
exponential_buckets(0.01, 10.0, 6).unwrap(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// ------- End of partition tree memtable metrics.
|
||||
|
||||
|
||||
// Manifest related metrics:
|
||||
|
||||
/// Elapsed time of manifest operation. Labeled with "op".
|
||||
|
||||
@@ -72,6 +72,41 @@ use crate::error::{
|
||||
Result,
|
||||
};
|
||||
use crate::memtable::{BoxedBatchIterator, BoxedRecordBatchIterator};
|
||||
|
||||
pub(crate) fn timestamp_array_to_i64_slice(arr: &ArrayRef) -> &[i64] {
|
||||
use datatypes::arrow::array::{
|
||||
TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
|
||||
TimestampSecondArray,
|
||||
};
|
||||
use datatypes::arrow::datatypes::{DataType, TimeUnit};
|
||||
|
||||
match arr.data_type() {
|
||||
DataType::Timestamp(t, _) => match t {
|
||||
TimeUnit::Second => arr
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampSecondArray>()
|
||||
.unwrap()
|
||||
.values(),
|
||||
TimeUnit::Millisecond => arr
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampMillisecondArray>()
|
||||
.unwrap()
|
||||
.values(),
|
||||
TimeUnit::Microsecond => arr
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampMicrosecondArray>()
|
||||
.unwrap()
|
||||
.values(),
|
||||
TimeUnit::Nanosecond => arr
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampNanosecondArray>()
|
||||
.unwrap()
|
||||
.values(),
|
||||
},
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Storage internal representation of a batch of rows for a primary key (time series).
|
||||
///
|
||||
/// Rows are sorted by primary key, timestamp, sequence desc, op_type desc. Fields
|
||||
|
||||
@@ -38,9 +38,9 @@ use futures::{Stream, TryStreamExt};
|
||||
use snafu::ResultExt;
|
||||
|
||||
use crate::error::{ComputeArrowSnafu, NewRecordBatchSnafu, Result};
|
||||
use crate::memtable::partition_tree::data::timestamp_array_to_i64_slice;
|
||||
use crate::metrics::MERGE_FILTER_ROWS_TOTAL;
|
||||
use crate::read::dedup::{DedupMetrics, DedupMetricsReport};
|
||||
use crate::read::timestamp_array_to_i64_slice;
|
||||
use crate::sst::parquet::flat_format::{
|
||||
op_type_column_index, primary_key_column_index, time_index_column_index,
|
||||
};
|
||||
|
||||
@@ -29,8 +29,9 @@ use crate::cache::{
|
||||
selector_result_cache_hit, selector_result_cache_miss,
|
||||
};
|
||||
use crate::error::{ComputeArrowSnafu, Result};
|
||||
use crate::memtable::partition_tree::data::timestamp_array_to_i64_slice;
|
||||
use crate::read::{Batch, BatchReader, BoxedBatchReader, BoxedRecordBatchStream};
|
||||
use crate::read::{
|
||||
Batch, BatchReader, BoxedBatchReader, BoxedRecordBatchStream, timestamp_array_to_i64_slice,
|
||||
};
|
||||
use crate::sst::parquet::DEFAULT_READ_BATCH_SIZE;
|
||||
use crate::sst::parquet::flat_format::{primary_key_column_index, time_index_column_index};
|
||||
use crate::sst::parquet::format::{PrimaryKeyArray, primary_key_offsets};
|
||||
|
||||
@@ -179,7 +179,8 @@ impl RegionOpener {
|
||||
|
||||
/// Parses and sets options for the region.
|
||||
pub(crate) fn parse_options(self, options: HashMap<String, String>) -> Result<Self> {
|
||||
self.options(RegionOptions::try_from(&options)?)
|
||||
let region_id = self.region_id;
|
||||
self.options(RegionOptions::try_from_options(region_id, &options)?)
|
||||
}
|
||||
|
||||
/// Sets the replay checkpoint for the region.
|
||||
@@ -615,18 +616,24 @@ pub(crate) fn version_builder_from_manifest(
|
||||
|
||||
/// Updates region options by persistent options.
|
||||
pub(crate) fn sanitize_region_options(manifest: &RegionManifest, options: &mut RegionOptions) {
|
||||
let option_format = options.sst_format.unwrap_or_default();
|
||||
if option_format != manifest.sst_format {
|
||||
common_telemetry::warn!(
|
||||
"Overriding SST format from {:?} to {:?} for region {}",
|
||||
option_format,
|
||||
manifest.sst_format,
|
||||
manifest.metadata.region_id,
|
||||
);
|
||||
// The caller-supplied options win when they specify an explicit `sst_format`,
|
||||
// so re-parsed options (e.g. bulk memtable forcing flat format) take effect on
|
||||
// the running region instead of being clobbered by the persisted manifest value.
|
||||
// Fall back to the manifest only when the caller did not provide a value.
|
||||
match options.sst_format {
|
||||
Some(format) if format != manifest.sst_format => {
|
||||
common_telemetry::warn!(
|
||||
"Overriding SST format from {:?} (manifest) to {:?} (options) for region {}",
|
||||
manifest.sst_format,
|
||||
format,
|
||||
manifest.metadata.region_id,
|
||||
);
|
||||
}
|
||||
Some(_) => {}
|
||||
None => {
|
||||
options.sst_format = Some(manifest.sst_format);
|
||||
}
|
||||
}
|
||||
// Always set sst_format from manifest to ensure it's explicitly stored,
|
||||
// even when the default matches the manifest value.
|
||||
options.sst_format = Some(manifest.sst_format);
|
||||
if let Some(manifest_append_mode) = manifest.append_mode
|
||||
&& options.append_mode != manifest_append_mode
|
||||
{
|
||||
@@ -1156,6 +1163,7 @@ fn can_load_cache(state: RegionRoleState) -> bool {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
@@ -1171,15 +1179,60 @@ mod tests {
|
||||
use store_api::region_request::PathType;
|
||||
use store_api::storage::{FileId, RegionId};
|
||||
|
||||
use super::preload_parquet_meta_cache_for_files;
|
||||
use super::{preload_parquet_meta_cache_for_files, sanitize_region_options};
|
||||
use crate::cache::CacheManager;
|
||||
use crate::cache::file_cache::{FileType, IndexKey};
|
||||
use crate::manifest::action::{RegionManifest, RemovedFilesRecord};
|
||||
use crate::region::options::RegionOptions;
|
||||
use crate::sst::FormatType;
|
||||
use crate::sst::file::{FileHandle, FileMeta};
|
||||
use crate::sst::file_purger::NoopFilePurger;
|
||||
use crate::sst::parquet::PARQUET_METADATA_KEY;
|
||||
use crate::test_util::TestEnv;
|
||||
use crate::test_util::sst_util::sst_region_metadata;
|
||||
|
||||
fn build_test_manifest(sst_format: FormatType) -> RegionManifest {
|
||||
RegionManifest {
|
||||
metadata: Arc::new(sst_region_metadata()),
|
||||
files: HashMap::new(),
|
||||
removed_files: RemovedFilesRecord::default(),
|
||||
flushed_entry_id: 0,
|
||||
flushed_sequence: 0,
|
||||
committed_sequence: None,
|
||||
manifest_version: 0,
|
||||
truncated_entry_id: None,
|
||||
compaction_time_window: None,
|
||||
sst_format,
|
||||
append_mode: None,
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_sanitize_region_options_options_format_wins() {
|
||||
// Manifest persisted PrimaryKey, but the re-parsed options now request Flat
|
||||
// (e.g., bulk memtable). The options' value must win.
|
||||
let manifest = build_test_manifest(FormatType::PrimaryKey);
|
||||
let mut options = RegionOptions {
|
||||
sst_format: Some(FormatType::Flat),
|
||||
..Default::default()
|
||||
};
|
||||
sanitize_region_options(&manifest, &mut options);
|
||||
assert_eq!(options.sst_format, Some(FormatType::Flat));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_sanitize_region_options_fills_from_manifest_when_unset() {
|
||||
// When the caller didn't specify sst_format, fall back to the manifest value
|
||||
// so downstream code always sees a concrete format.
|
||||
let manifest = build_test_manifest(FormatType::Flat);
|
||||
let mut options = RegionOptions {
|
||||
sst_format: None,
|
||||
..Default::default()
|
||||
};
|
||||
sanitize_region_options(&manifest, &mut options);
|
||||
assert_eq!(options.sst_format, Some(FormatType::Flat));
|
||||
}
|
||||
|
||||
fn sst_parquet_bytes(batch: &RecordBatch) -> Vec<u8> {
|
||||
let key_value_meta = KeyValue::new(
|
||||
PARQUET_METADATA_KEY.to_string(),
|
||||
|
||||
@@ -20,7 +20,7 @@ use std::collections::HashMap;
|
||||
use std::time::Duration;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
use common_stat::get_total_memory_readable;
|
||||
use common_telemetry::info;
|
||||
use common_time::TimeToLive;
|
||||
use common_wal::options::{WAL_OPTIONS_KEY, WalOptions};
|
||||
use serde::de::Error as _;
|
||||
@@ -29,17 +29,26 @@ use serde_json::Value;
|
||||
use serde_with::{DisplayFromStr, NoneAsEmptyString, serde_as, with_prefix};
|
||||
use snafu::{ResultExt, ensure};
|
||||
use store_api::codec::PrimaryKeyEncoding;
|
||||
use store_api::metric_engine_consts::{
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING, PRIMARY_KEY_ENCODING,
|
||||
};
|
||||
use store_api::mito_engine_options::COMPACTION_OVERRIDE;
|
||||
use store_api::storage::ColumnId;
|
||||
use store_api::storage::{ColumnId, RegionId};
|
||||
use strum::EnumString;
|
||||
|
||||
use crate::error::{Error, InvalidRegionOptionsSnafu, JsonOptionsSnafu, Result};
|
||||
use crate::memtable::partition_tree::{DEFAULT_FREEZE_THRESHOLD, DEFAULT_MAX_KEYS_PER_SHARD};
|
||||
use crate::error::{InvalidRegionOptionsSnafu, JsonOptionsSnafu, Result};
|
||||
use crate::memtable::bulk::BulkMemtableConfig;
|
||||
use crate::sst::FormatType;
|
||||
|
||||
const DEFAULT_INDEX_SEGMENT_ROW_COUNT: usize = 1024;
|
||||
const COMPACTION_TWCS_PREFIX: &str = "compaction.twcs.";
|
||||
const MEMTABLE_PARTITION_TREE_PREFIX: &str = "memtable.partition_tree.";
|
||||
const MEMTABLE_BULK_PREFIX: &str = "memtable.bulk.";
|
||||
|
||||
/// Legacy memtable type identifier accepted for backward compatibility.
|
||||
/// The partition tree memtable has been removed; parsing this value falls
|
||||
/// back to the default (bulk) memtable at runtime.
|
||||
const LEGACY_PARTITION_TREE_MEMTABLE_TYPE: &str = "partition_tree";
|
||||
|
||||
pub(crate) fn parse_wal_options(
|
||||
options_map: &HashMap<String, String>,
|
||||
@@ -91,6 +100,9 @@ pub struct RegionOptions {
|
||||
pub merge_mode: Option<MergeMode>,
|
||||
/// SST format type.
|
||||
pub sst_format: Option<FormatType>,
|
||||
/// Internal primary key encoding override used by metric-engine.
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
pub primary_key_encoding: Option<PrimaryKeyEncoding>,
|
||||
}
|
||||
|
||||
impl RegionOptions {
|
||||
@@ -120,18 +132,16 @@ impl RegionOptions {
|
||||
|
||||
/// Returns the `primary_key_encoding` if it is set, otherwise returns the default [`PrimaryKeyEncoding`].
|
||||
pub fn primary_key_encoding(&self) -> PrimaryKeyEncoding {
|
||||
self.memtable
|
||||
.as_ref()
|
||||
.map_or(PrimaryKeyEncoding::default(), |memtable| {
|
||||
memtable.primary_key_encoding()
|
||||
})
|
||||
self.primary_key_encoding.unwrap_or_default()
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<&HashMap<String, String>> for RegionOptions {
|
||||
type Error = Error;
|
||||
|
||||
fn try_from(options_map: &HashMap<String, String>) -> Result<Self> {
|
||||
impl RegionOptions {
|
||||
/// Parses [RegionOptions] from the raw `options_map`.
|
||||
pub(crate) fn try_from_options(
|
||||
region_id: RegionId,
|
||||
options_map: &HashMap<String, String>,
|
||||
) -> Result<Self> {
|
||||
let value = options_map_to_value(options_map);
|
||||
let json = serde_json::to_string(&value).context(JsonOptionsSnafu)?;
|
||||
|
||||
@@ -151,21 +161,70 @@ impl TryFrom<&HashMap<String, String>> for RegionOptions {
|
||||
let wal_options = parse_wal_options(options_map).context(JsonOptionsSnafu)?;
|
||||
|
||||
let index_options: IndexOptions = serde_json::from_str(&json).context(JsonOptionsSnafu)?;
|
||||
let is_legacy_partition_tree = options_map
|
||||
.get("memtable.type")
|
||||
.map(|s| s.eq_ignore_ascii_case(LEGACY_PARTITION_TREE_MEMTABLE_TYPE))
|
||||
.unwrap_or(false);
|
||||
let memtable = if validate_enum_options(
|
||||
options_map,
|
||||
"memtable.type",
|
||||
&[MEMTABLE_PARTITION_TREE_PREFIX],
|
||||
&[MEMTABLE_PARTITION_TREE_PREFIX, MEMTABLE_BULK_PREFIX],
|
||||
)? {
|
||||
Some(serde_json::from_str(&json).context(JsonOptionsSnafu)?)
|
||||
if is_legacy_partition_tree {
|
||||
// The partition tree memtable has been removed. Fall back to the
|
||||
// default memtable; the primary key encoding (if any) is still
|
||||
// read separately below from the legacy nested key.
|
||||
None
|
||||
} else {
|
||||
Some(serde_json::from_str(&json).context(JsonOptionsSnafu)?)
|
||||
}
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
// The partition tree memtable has been removed. Besides falling back to
|
||||
// the default memtable, also override the SST format to flat.
|
||||
let mut sst_format = options.sst_format;
|
||||
if is_legacy_partition_tree {
|
||||
info!(
|
||||
"Region {} specified the removed partition_tree memtable; \
|
||||
overriding memtable to the default and SST format to flat",
|
||||
region_id
|
||||
);
|
||||
sst_format = Some(FormatType::Flat);
|
||||
}
|
||||
|
||||
// Bulk memtable produces flat-encoded ranges and flushes them through
|
||||
// `put_sst()`, so the SST format must be flat to match.
|
||||
if matches!(memtable, Some(MemtableOptions::Bulk(_))) {
|
||||
if let Some(format) = sst_format
|
||||
&& format != FormatType::Flat
|
||||
{
|
||||
info!(
|
||||
"Region {} uses bulk memtable; overriding sst_format from {:?} to flat",
|
||||
region_id, format
|
||||
);
|
||||
}
|
||||
sst_format = Some(FormatType::Flat);
|
||||
}
|
||||
|
||||
let compaction_override_flag = options_map
|
||||
.get(COMPACTION_OVERRIDE)
|
||||
.map(|v| matches!(v.to_lowercase().as_str(), "true" | "1"))
|
||||
.unwrap_or(false);
|
||||
let compaction_override = has_compaction_type || compaction_override_flag;
|
||||
let primary_key_encoding = options_map
|
||||
.get(PRIMARY_KEY_ENCODING)
|
||||
.or_else(|| options_map.get(MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING))
|
||||
.map(|v| match v.to_lowercase().as_str() {
|
||||
"dense" => Ok(PrimaryKeyEncoding::Dense),
|
||||
"sparse" => Ok(PrimaryKeyEncoding::Sparse),
|
||||
_ => Err(InvalidRegionOptionsSnafu {
|
||||
reason: format!("Invalid primary key encoding: {v}"),
|
||||
}
|
||||
.build()),
|
||||
})
|
||||
.transpose()?;
|
||||
|
||||
let opts = RegionOptions {
|
||||
ttl: options.ttl,
|
||||
@@ -177,7 +236,8 @@ impl TryFrom<&HashMap<String, String>> for RegionOptions {
|
||||
index_options,
|
||||
memtable,
|
||||
merge_mode: options.merge_mode,
|
||||
sst_format: options.sst_format,
|
||||
sst_format,
|
||||
primary_key_encoding,
|
||||
};
|
||||
opts.validate()?;
|
||||
|
||||
@@ -341,59 +401,11 @@ impl Default for InvertedIndexOptions {
|
||||
#[serde(tag = "memtable.type", rename_all = "snake_case")]
|
||||
pub enum MemtableOptions {
|
||||
TimeSeries,
|
||||
#[serde(with = "prefix_partition_tree")]
|
||||
PartitionTree(PartitionTreeOptions),
|
||||
#[serde(with = "prefix_bulk")]
|
||||
Bulk(BulkMemtableConfig),
|
||||
}
|
||||
|
||||
with_prefix!(prefix_partition_tree "memtable.partition_tree.");
|
||||
|
||||
impl MemtableOptions {
|
||||
/// Returns the primary key encoding mode.
|
||||
pub fn primary_key_encoding(&self) -> PrimaryKeyEncoding {
|
||||
match self {
|
||||
MemtableOptions::PartitionTree(opts) => opts.primary_key_encoding,
|
||||
_ => PrimaryKeyEncoding::Dense,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Partition tree memtable options.
|
||||
#[serde_as]
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
|
||||
#[serde(default)]
|
||||
pub struct PartitionTreeOptions {
|
||||
/// Max keys in an index shard.
|
||||
#[serde_as(as = "DisplayFromStr")]
|
||||
pub index_max_keys_per_shard: usize,
|
||||
/// Number of rows to freeze a data part.
|
||||
#[serde_as(as = "DisplayFromStr")]
|
||||
pub data_freeze_threshold: usize,
|
||||
/// Total bytes of dictionary to keep in fork.
|
||||
pub fork_dictionary_bytes: ReadableSize,
|
||||
/// Primary key encoding mode.
|
||||
pub primary_key_encoding: PrimaryKeyEncoding,
|
||||
}
|
||||
|
||||
impl Default for PartitionTreeOptions {
|
||||
fn default() -> Self {
|
||||
let mut fork_dictionary_bytes = ReadableSize::mb(512);
|
||||
if let Some(total_memory) = get_total_memory_readable() {
|
||||
let adjust_dictionary_bytes = std::cmp::min(
|
||||
total_memory / crate::memtable::partition_tree::DICTIONARY_SIZE_FACTOR,
|
||||
fork_dictionary_bytes,
|
||||
);
|
||||
if adjust_dictionary_bytes.0 > 0 {
|
||||
fork_dictionary_bytes = adjust_dictionary_bytes;
|
||||
}
|
||||
}
|
||||
Self {
|
||||
index_max_keys_per_shard: DEFAULT_MAX_KEYS_PER_SHARD,
|
||||
data_freeze_threshold: DEFAULT_FREEZE_THRESHOLD,
|
||||
fork_dictionary_bytes,
|
||||
primary_key_encoding: PrimaryKeyEncoding::Dense,
|
||||
}
|
||||
}
|
||||
}
|
||||
with_prefix!(prefix_bulk "memtable.bulk.");
|
||||
|
||||
fn deserialize_ignore_column_ids<'de, D>(deserializer: D) -> Result<Vec<ColumnId>, D::Error>
|
||||
where
|
||||
@@ -500,14 +512,14 @@ mod tests {
|
||||
#[test]
|
||||
fn test_empty_region_options() {
|
||||
let map = make_map(&[]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert_eq!(RegionOptions::default(), options);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_with_ttl() {
|
||||
let map = make_map(&[("ttl", "7d")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
ttl: Some(Duration::from_secs(3600 * 24 * 7).into()),
|
||||
..Default::default()
|
||||
@@ -518,7 +530,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_with_storage() {
|
||||
let map = make_map(&[("storage", "S3")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
storage: Some("S3".to_string()),
|
||||
..Default::default()
|
||||
@@ -532,7 +544,7 @@ mod tests {
|
||||
("compaction.twcs.trigger_file_num", "8"),
|
||||
("compaction.twcs.time_window", "2h"),
|
||||
]);
|
||||
let err = RegionOptions::try_from(&map).unwrap_err();
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
@@ -543,7 +555,7 @@ mod tests {
|
||||
("compaction.twcs.time_window", "2h"),
|
||||
("compaction.type", "twcs"),
|
||||
]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
compaction: CompactionOptions::Twcs(TwcsOptions {
|
||||
trigger_file_num: 8,
|
||||
@@ -559,7 +571,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_with_compaction_override_true_without_compaction_type() {
|
||||
let map = make_map(&[(COMPACTION_OVERRIDE, "true")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
compaction_override: true,
|
||||
..Default::default()
|
||||
@@ -570,7 +582,7 @@ mod tests {
|
||||
#[test]
|
||||
fn test_with_compaction_override_false_without_compaction_type() {
|
||||
let map = make_map(&[(COMPACTION_OVERRIDE, "false")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert_eq!(RegionOptions::default(), options);
|
||||
}
|
||||
|
||||
@@ -580,14 +592,14 @@ mod tests {
|
||||
(COMPACTION_OVERRIDE, "true"),
|
||||
("compaction.twcs.time_window", "2h"),
|
||||
]);
|
||||
let err = RegionOptions::try_from(&map).unwrap_err();
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
fn test_with_wal_options(wal_options: &WalOptions) -> bool {
|
||||
let encoded_wal_options = serde_json::to_string(&wal_options).unwrap();
|
||||
let map = make_map(&[(WAL_OPTIONS_KEY, &encoded_wal_options)]);
|
||||
let got = RegionOptions::try_from(&map).unwrap();
|
||||
let got = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
wal_options: wal_options.clone(),
|
||||
..Default::default()
|
||||
@@ -601,7 +613,7 @@ mod tests {
|
||||
("index.inverted_index.ignore_column_ids", "1,2,3"),
|
||||
("index.inverted_index.segment_row_count", "512"),
|
||||
]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
index_options: IndexOptions {
|
||||
inverted_index: InvertedIndexOptions {
|
||||
@@ -629,62 +641,164 @@ mod tests {
|
||||
#[test]
|
||||
fn test_with_memtable() {
|
||||
let map = make_map(&[("memtable.type", "time_series")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
memtable: Some(MemtableOptions::TimeSeries),
|
||||
..Default::default()
|
||||
};
|
||||
assert_eq!(expect, options);
|
||||
|
||||
let map = make_map(&[("memtable.type", "partition_tree")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let map = make_map(&[("memtable.type", "bulk")]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
memtable: Some(MemtableOptions::PartitionTree(
|
||||
PartitionTreeOptions::default(),
|
||||
)),
|
||||
memtable: Some(MemtableOptions::Bulk(BulkMemtableConfig::default())),
|
||||
sst_format: Some(FormatType::Flat),
|
||||
..Default::default()
|
||||
};
|
||||
assert_eq!(expect, options);
|
||||
|
||||
let map = make_map(&[
|
||||
("memtable.type", "bulk"),
|
||||
("memtable.bulk.merge_threshold", "7"),
|
||||
("memtable.bulk.encode_row_threshold", "11"),
|
||||
("memtable.bulk.encode_bytes_threshold", "13"),
|
||||
("memtable.bulk.max_merge_groups", "17"),
|
||||
]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
memtable: Some(MemtableOptions::Bulk(BulkMemtableConfig {
|
||||
merge_threshold: 7,
|
||||
encode_row_threshold: 11,
|
||||
encode_bytes_threshold: 13,
|
||||
max_merge_groups: 17,
|
||||
})),
|
||||
sst_format: Some(FormatType::Flat),
|
||||
..Default::default()
|
||||
};
|
||||
assert_eq!(expect, options);
|
||||
|
||||
// Legacy partition_tree memtable falls back to the default memtable and
|
||||
// overrides the SST format to flat.
|
||||
let map = make_map(&[("memtable.type", "partition_tree")]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
memtable: None,
|
||||
sst_format: Some(FormatType::Flat),
|
||||
..Default::default()
|
||||
};
|
||||
assert_eq!(expect, options);
|
||||
|
||||
// Legacy partition_tree options are tolerated alongside the type tag.
|
||||
let map = make_map(&[
|
||||
("memtable.type", "partition_tree"),
|
||||
("memtable.partition_tree.index_max_keys_per_shard", "2048"),
|
||||
("memtable.partition_tree.fork_dictionary_bytes", "128M"),
|
||||
]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
memtable: None,
|
||||
sst_format: Some(FormatType::Flat),
|
||||
..Default::default()
|
||||
};
|
||||
assert_eq!(expect, options);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_primary_key_encoding() {
|
||||
// New top-level key.
|
||||
let map = make_map(&[("primary_key_encoding", "sparse")]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert_eq!(options.primary_key_encoding(), PrimaryKeyEncoding::Sparse);
|
||||
assert_eq!(
|
||||
options.primary_key_encoding,
|
||||
Some(PrimaryKeyEncoding::Sparse)
|
||||
);
|
||||
|
||||
// Legacy memtable.type=partition_tree + legacy encoding.
|
||||
let map = make_map(&[
|
||||
("memtable.type", "partition_tree"),
|
||||
("memtable.partition_tree.primary_key_encoding", "sparse"),
|
||||
]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert_eq!(options.memtable, None);
|
||||
assert_eq!(options.sst_format, Some(FormatType::Flat));
|
||||
assert_eq!(options.primary_key_encoding(), PrimaryKeyEncoding::Sparse);
|
||||
|
||||
// Invalid value rejected.
|
||||
let map = make_map(&[("primary_key_encoding", "bogus")]);
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_legacy_partition_tree_overrides_sst_format() {
|
||||
// Legacy partition_tree memtable falls back to the default memtable and
|
||||
// overrides the SST format to flat, even when a different format was set.
|
||||
let map = make_map(&[
|
||||
("memtable.type", "partition_tree"),
|
||||
("sst_format", "primary_key"),
|
||||
]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(1, 1), &map).unwrap();
|
||||
assert_eq!(options.memtable, None);
|
||||
assert_eq!(options.sst_format, Some(FormatType::Flat));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_bulk_memtable_overrides_sst_format() {
|
||||
// Bulk memtable produces flat-encoded ranges, so an explicit
|
||||
// `sst_format=primary_key` must be overridden to flat to keep the
|
||||
// in-memory and on-disk encodings in sync.
|
||||
let map = make_map(&[("memtable.type", "bulk"), ("sst_format", "primary_key")]);
|
||||
let options = RegionOptions::try_from_options(RegionId::new(1, 1), &map).unwrap();
|
||||
assert_eq!(
|
||||
options.memtable,
|
||||
Some(MemtableOptions::Bulk(BulkMemtableConfig::default()))
|
||||
);
|
||||
assert_eq!(options.sst_format, Some(FormatType::Flat));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_unknown_memtable_type() {
|
||||
let map = make_map(&[("memtable.type", "no_such_memtable")]);
|
||||
let err = RegionOptions::try_from(&map).unwrap_err();
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_without_memtable_type() {
|
||||
let map = make_map(&[("memtable.partition_tree.index_max_keys_per_shard", "2048")]);
|
||||
let err = RegionOptions::try_from(&map).unwrap_err();
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
|
||||
let map = make_map(&[("memtable.bulk.merge_threshold", "7")]);
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_with_merge_mode() {
|
||||
let map = make_map(&[("merge_mode", "last_row")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert_eq!(MergeMode::LastRow, options.merge_mode());
|
||||
|
||||
let map = make_map(&[("merge_mode", "last_non_null")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert_eq!(MergeMode::LastNonNull, options.merge_mode());
|
||||
|
||||
let map = make_map(&[("merge_mode", "unknown")]);
|
||||
let err = RegionOptions::try_from(&map).unwrap_err();
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_append_mode_allows_last_row_merge_mode() {
|
||||
let map = make_map(&[("append_mode", "true"), ("merge_mode", "last_row")]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
assert!(options.append_mode);
|
||||
assert_eq!(MergeMode::LastRow, options.merge_mode());
|
||||
|
||||
let map = make_map(&[("append_mode", "true"), ("merge_mode", "last_non_null")]);
|
||||
let err = RegionOptions::try_from(&map).unwrap_err();
|
||||
let err = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap_err();
|
||||
assert_eq!(StatusCode::InvalidArguments, err.status_code());
|
||||
}
|
||||
|
||||
@@ -709,13 +823,14 @@ mod tests {
|
||||
WAL_OPTIONS_KEY,
|
||||
&serde_json::to_string(&wal_options).unwrap(),
|
||||
),
|
||||
("memtable.type", "partition_tree"),
|
||||
("memtable.partition_tree.index_max_keys_per_shard", "2048"),
|
||||
("memtable.partition_tree.data_freeze_threshold", "2048"),
|
||||
("memtable.partition_tree.fork_dictionary_bytes", "128M"),
|
||||
("memtable.type", "bulk"),
|
||||
("memtable.bulk.merge_threshold", "7"),
|
||||
("memtable.bulk.encode_row_threshold", "11"),
|
||||
("memtable.bulk.encode_bytes_threshold", "13"),
|
||||
("memtable.bulk.max_merge_groups", "17"),
|
||||
("merge_mode", "last_non_null"),
|
||||
]);
|
||||
let options = RegionOptions::try_from(&map).unwrap();
|
||||
let options = RegionOptions::try_from_options(RegionId::new(0, 0), &map).unwrap();
|
||||
let expect = RegionOptions {
|
||||
ttl: Some(Duration::from_secs(3600 * 24 * 7).into()),
|
||||
compaction: CompactionOptions::Twcs(TwcsOptions {
|
||||
@@ -735,14 +850,15 @@ mod tests {
|
||||
segment_row_count: 512,
|
||||
},
|
||||
},
|
||||
memtable: Some(MemtableOptions::PartitionTree(PartitionTreeOptions {
|
||||
index_max_keys_per_shard: 2048,
|
||||
data_freeze_threshold: 2048,
|
||||
fork_dictionary_bytes: ReadableSize::mb(128),
|
||||
primary_key_encoding: PrimaryKeyEncoding::Dense,
|
||||
memtable: Some(MemtableOptions::Bulk(BulkMemtableConfig {
|
||||
merge_threshold: 7,
|
||||
encode_row_threshold: 11,
|
||||
encode_bytes_threshold: 13,
|
||||
max_merge_groups: 17,
|
||||
})),
|
||||
merge_mode: Some(MergeMode::LastNonNull),
|
||||
sst_format: None,
|
||||
sst_format: Some(FormatType::Flat),
|
||||
primary_key_encoding: None,
|
||||
};
|
||||
assert_eq!(expect, options);
|
||||
}
|
||||
@@ -770,14 +886,10 @@ mod tests {
|
||||
segment_row_count: 512,
|
||||
},
|
||||
},
|
||||
memtable: Some(MemtableOptions::PartitionTree(PartitionTreeOptions {
|
||||
index_max_keys_per_shard: 2048,
|
||||
data_freeze_threshold: 2048,
|
||||
fork_dictionary_bytes: ReadableSize::mb(128),
|
||||
primary_key_encoding: PrimaryKeyEncoding::Dense,
|
||||
})),
|
||||
memtable: Some(MemtableOptions::Bulk(BulkMemtableConfig::default())),
|
||||
merge_mode: Some(MergeMode::LastNonNull),
|
||||
sst_format: None,
|
||||
primary_key_encoding: None,
|
||||
};
|
||||
let region_options_json_str = serde_json::to_string(&options).unwrap();
|
||||
let got: RegionOptions = serde_json::from_str(®ion_options_json_str).unwrap();
|
||||
@@ -806,10 +918,7 @@ mod tests {
|
||||
"index.inverted_index.segment_row_count": "512"
|
||||
},
|
||||
"memtable": {
|
||||
"memtable.type": "partition_tree",
|
||||
"memtable.partition_tree.index_max_keys_per_shard": "2048",
|
||||
"memtable.partition_tree.data_freeze_threshold": "2048",
|
||||
"memtable.partition_tree.fork_dictionary_bytes": "128MiB"
|
||||
"memtable.type": "bulk"
|
||||
},
|
||||
"merge_mode": "last_non_null"
|
||||
}"#;
|
||||
@@ -835,14 +944,10 @@ mod tests {
|
||||
segment_row_count: 512,
|
||||
},
|
||||
},
|
||||
memtable: Some(MemtableOptions::PartitionTree(PartitionTreeOptions {
|
||||
index_max_keys_per_shard: 2048,
|
||||
data_freeze_threshold: 2048,
|
||||
fork_dictionary_bytes: ReadableSize::mb(128),
|
||||
primary_key_encoding: PrimaryKeyEncoding::Dense,
|
||||
})),
|
||||
memtable: Some(MemtableOptions::Bulk(BulkMemtableConfig::default())),
|
||||
merge_mode: Some(MergeMode::LastNonNull),
|
||||
sst_format: None,
|
||||
primary_key_encoding: None,
|
||||
};
|
||||
assert_eq!(options, got);
|
||||
}
|
||||
|
||||
@@ -499,9 +499,9 @@ impl PrefilterContextBuilder {
|
||||
/// Builds a [PrefilterContext] for a specific row group.
|
||||
pub(crate) fn build(&self) -> PrefilterContext {
|
||||
let pk_filter = self.pk_filters.as_ref().map(|pk_filters| {
|
||||
let pk_filter =
|
||||
self.codec
|
||||
.primary_key_filter(&self.metadata, Arc::clone(pk_filters), false);
|
||||
let pk_filter = self
|
||||
.codec
|
||||
.primary_key_filter(&self.metadata, Arc::clone(pk_filters));
|
||||
Box::new(CachedPrimaryKeyFilter::new(pk_filter)) as Box<dyn PrimaryKeyFilter>
|
||||
});
|
||||
PrefilterContext {
|
||||
@@ -927,8 +927,8 @@ mod tests {
|
||||
fn test_prefilter_primary_key_drops_single_dictionary_batch() {
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
let filters = Arc::new(new_test_filters(&[col("tag_0").eq(lit("b"))]));
|
||||
let mut primary_key_filter = build_primary_key_codec(metadata.as_ref())
|
||||
.primary_key_filter(&metadata, filters, false);
|
||||
let mut primary_key_filter =
|
||||
build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters);
|
||||
let pk_a = new_primary_key(&["a", "x"]);
|
||||
let batch = new_raw_batch(&[pk_a.as_slice(), pk_a.as_slice()], &[10, 11]);
|
||||
let pk_col_idx = primary_key_column_index(batch.num_columns());
|
||||
@@ -946,8 +946,8 @@ mod tests {
|
||||
let filters = Arc::new(new_test_filters(&[col("tag_0")
|
||||
.eq(lit("a"))
|
||||
.or(col("tag_0").eq(lit("c")))]));
|
||||
let mut primary_key_filter = build_primary_key_codec(metadata.as_ref())
|
||||
.primary_key_filter(&metadata, filters, false);
|
||||
let mut primary_key_filter =
|
||||
build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters);
|
||||
let pk_a = new_primary_key(&["a", "x"]);
|
||||
let pk_b = new_primary_key(&["b", "x"]);
|
||||
let pk_c = new_primary_key(&["c", "x"]);
|
||||
@@ -1196,8 +1196,7 @@ mod tests {
|
||||
let metadata = Arc::new(sst_region_metadata());
|
||||
let filters = Arc::new(new_test_filters(&[col("tag_0").eq(lit("a"))]));
|
||||
let mut pk_filter = Some(Box::new(CachedPrimaryKeyFilter::new(
|
||||
build_primary_key_codec(metadata.as_ref())
|
||||
.primary_key_filter(&metadata, filters, false),
|
||||
build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters),
|
||||
)) as Box<dyn PrimaryKeyFilter>);
|
||||
let pk_a = new_primary_key(&["a", "x"]);
|
||||
let pk_b = new_primary_key(&["b", "x"]);
|
||||
|
||||
@@ -20,13 +20,11 @@ use api::helper::ColumnDataTypeWrapper;
|
||||
use api::v1::value::ValueData;
|
||||
use api::v1::{Row, Rows, SemanticType};
|
||||
use common_time::Timestamp;
|
||||
use datatypes::arrow::array::UInt64Array;
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::scalars::ScalarVector;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
use datatypes::vectors::TimestampMillisecondVector;
|
||||
use mito_codec::key_values::KeyValue;
|
||||
use mito_codec::row_converter::{DensePrimaryKeyCodec, PrimaryKeyCodecExt, SortField};
|
||||
use store_api::metadata::{
|
||||
ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef,
|
||||
};
|
||||
@@ -34,7 +32,6 @@ use store_api::storage::{ColumnId, RegionId, SequenceNumber};
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::memtable::bulk::part::BulkPart;
|
||||
use crate::memtable::partition_tree::data::{DataBatch, DataBuffer, timestamp_array_to_i64_slice};
|
||||
use crate::memtable::{
|
||||
BoxedBatchIterator, KeyValues, Memtable, MemtableBuilder, MemtableId, MemtableRanges,
|
||||
MemtableRef, MemtableStats, RangesOptions,
|
||||
@@ -204,46 +201,6 @@ pub fn build_key_values(
|
||||
)
|
||||
}
|
||||
|
||||
pub(crate) fn write_rows_to_buffer(
|
||||
buffer: &mut DataBuffer,
|
||||
schema: &RegionMetadataRef,
|
||||
pk_index: u16,
|
||||
ts: Vec<i64>,
|
||||
v0: Vec<Option<f64>>,
|
||||
sequence: u64,
|
||||
) {
|
||||
let kvs = crate::test_util::memtable_util::build_key_values_with_ts_seq_values(
|
||||
schema,
|
||||
"whatever".to_string(),
|
||||
1,
|
||||
ts.into_iter(),
|
||||
v0.into_iter(),
|
||||
sequence,
|
||||
);
|
||||
|
||||
for kv in kvs.iter() {
|
||||
buffer.write_row(pk_index, &kv);
|
||||
}
|
||||
}
|
||||
|
||||
/// Extracts pk index, timestamps and sequences from [DataBatch].
|
||||
pub(crate) fn extract_data_batch(batch: &DataBatch) -> (u16, Vec<(i64, u64)>) {
|
||||
let rb = batch.slice_record_batch();
|
||||
let ts = timestamp_array_to_i64_slice(rb.column(1));
|
||||
let seq = rb
|
||||
.column(2)
|
||||
.as_any()
|
||||
.downcast_ref::<UInt64Array>()
|
||||
.unwrap()
|
||||
.values();
|
||||
let ts_and_seq = ts
|
||||
.iter()
|
||||
.zip(seq.iter())
|
||||
.map(|(ts, seq)| (*ts, *seq))
|
||||
.collect::<Vec<_>>();
|
||||
(batch.pk_index(), ts_and_seq)
|
||||
}
|
||||
|
||||
/// Builds key values with timestamps (ms) and sequences for test.
|
||||
pub(crate) fn build_key_values_with_ts_seq_values(
|
||||
metadata: &RegionMetadataRef,
|
||||
@@ -305,28 +262,6 @@ pub fn region_metadata_to_row_schema(metadata: &RegionMetadataRef) -> Vec<api::v
|
||||
.collect()
|
||||
}
|
||||
|
||||
/// Encode keys.
|
||||
pub(crate) fn encode_keys(
|
||||
metadata: &RegionMetadataRef,
|
||||
key_values: &KeyValues,
|
||||
keys: &mut Vec<Vec<u8>>,
|
||||
) {
|
||||
let row_codec = DensePrimaryKeyCodec::new(metadata);
|
||||
for kv in key_values.iter() {
|
||||
let key = row_codec.encode(kv.primary_keys()).unwrap();
|
||||
keys.push(key);
|
||||
}
|
||||
}
|
||||
|
||||
/// Encode one key.
|
||||
pub(crate) fn encode_key_by_kv(key_value: &KeyValue) -> Vec<u8> {
|
||||
let row_codec = DensePrimaryKeyCodec::with_fields(vec![
|
||||
(0, SortField::new(ConcreteDataType::string_datatype())),
|
||||
(1, SortField::new(ConcreteDataType::uint32_datatype())),
|
||||
]);
|
||||
row_codec.encode(key_value.primary_keys()).unwrap()
|
||||
}
|
||||
|
||||
/// Collects timestamps from the batch iter.
|
||||
pub(crate) fn collect_iter_timestamps(iter: BoxedBatchIterator) -> Vec<i64> {
|
||||
iter.flat_map(|batch| {
|
||||
|
||||
@@ -130,7 +130,11 @@ pub const METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION_DEFAULT: u32 = 1
|
||||
/// Default false positive rate for the skipping index in the metric engine.
|
||||
pub const METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION_DEFAULT: f64 = 0.01;
|
||||
|
||||
/// Option key for memtable partition tree primary key encoding.
|
||||
/// Top-level region option key for primary key encoding. Metric-engine internal use.
|
||||
pub const PRIMARY_KEY_ENCODING: &str = "primary_key_encoding";
|
||||
|
||||
/// Deprecated. Kept for backward compatibility: parsing still accepts this key
|
||||
/// and migrates the value to [`PRIMARY_KEY_ENCODING`].
|
||||
pub const MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING: &str =
|
||||
"memtable.partition_tree.primary_key_encoding";
|
||||
|
||||
@@ -143,6 +147,7 @@ pub fn is_metric_engine_option_key(key: &str) -> bool {
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_GRANULARITY_OPTION,
|
||||
METRIC_ENGINE_INDEX_SKIPPING_INDEX_FALSE_POSITIVE_RATE_OPTION,
|
||||
// Only allow setting primary key encoding by metric engine.
|
||||
PRIMARY_KEY_ENCODING,
|
||||
MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING,
|
||||
]
|
||||
.contains(&key)
|
||||
|
||||
@@ -43,6 +43,14 @@ pub const TWCS_REMOTE_COMPACTION: &str = "compaction.twcs.remote_compaction";
|
||||
pub const TWCS_FALLBACK_TO_LOCAL: &str = "compaction.twcs.fallback_to_local";
|
||||
/// Option key for memtable type.
|
||||
pub const MEMTABLE_TYPE: &str = "memtable.type";
|
||||
/// Option key for bulk memtable merge threshold.
|
||||
pub const MEMTABLE_BULK_MERGE_THRESHOLD: &str = "memtable.bulk.merge_threshold";
|
||||
/// Option key for bulk memtable encode row threshold.
|
||||
pub const MEMTABLE_BULK_ENCODE_ROW_THRESHOLD: &str = "memtable.bulk.encode_row_threshold";
|
||||
/// Option key for bulk memtable encode bytes threshold.
|
||||
pub const MEMTABLE_BULK_ENCODE_BYTES_THRESHOLD: &str = "memtable.bulk.encode_bytes_threshold";
|
||||
/// Option key for bulk memtable max merge groups.
|
||||
pub const MEMTABLE_BULK_MAX_MERGE_GROUPS: &str = "memtable.bulk.max_merge_groups";
|
||||
/// Option key for memtable partition tree index max keys per shard.
|
||||
pub const MEMTABLE_PARTITION_TREE_INDEX_MAX_KEYS_PER_SHARD: &str =
|
||||
"memtable.partition_tree.index_max_keys_per_shard";
|
||||
@@ -74,6 +82,10 @@ pub fn is_mito_engine_option_key(key: &str) -> bool {
|
||||
"index.inverted_index.segment_row_count",
|
||||
WAL_OPTIONS_KEY,
|
||||
MEMTABLE_TYPE,
|
||||
MEMTABLE_BULK_MERGE_THRESHOLD,
|
||||
MEMTABLE_BULK_ENCODE_ROW_THRESHOLD,
|
||||
MEMTABLE_BULK_ENCODE_BYTES_THRESHOLD,
|
||||
MEMTABLE_BULK_MAX_MERGE_GROUPS,
|
||||
MEMTABLE_PARTITION_TREE_INDEX_MAX_KEYS_PER_SHARD,
|
||||
MEMTABLE_PARTITION_TREE_DATA_FREEZE_THRESHOLD,
|
||||
MEMTABLE_PARTITION_TREE_FORK_DICTIONARY_BYTES,
|
||||
@@ -107,6 +119,14 @@ mod tests {
|
||||
));
|
||||
assert!(is_mito_engine_option_key("wal_options"));
|
||||
assert!(is_mito_engine_option_key("memtable.type"));
|
||||
assert!(is_mito_engine_option_key("memtable.bulk.merge_threshold"));
|
||||
assert!(is_mito_engine_option_key(
|
||||
"memtable.bulk.encode_row_threshold"
|
||||
));
|
||||
assert!(is_mito_engine_option_key(
|
||||
"memtable.bulk.encode_bytes_threshold"
|
||||
));
|
||||
assert!(is_mito_engine_option_key("memtable.bulk.max_merge_groups"));
|
||||
assert!(is_mito_engine_option_key(
|
||||
"memtable.partition_tree.index_max_keys_per_shard"
|
||||
));
|
||||
|
||||
@@ -36,7 +36,9 @@ use store_api::metric_engine_consts::{
|
||||
LOGICAL_TABLE_METADATA_KEY, PHYSICAL_TABLE_METADATA_KEY, is_metric_engine_option_key,
|
||||
};
|
||||
use store_api::mito_engine_options::{
|
||||
APPEND_MODE_KEY, COMPACTION_TYPE, MEMTABLE_TYPE, MERGE_MODE_KEY, SST_FORMAT_KEY,
|
||||
APPEND_MODE_KEY, COMPACTION_TYPE, MEMTABLE_BULK_ENCODE_BYTES_THRESHOLD,
|
||||
MEMTABLE_BULK_ENCODE_ROW_THRESHOLD, MEMTABLE_BULK_MAX_MERGE_GROUPS,
|
||||
MEMTABLE_BULK_MERGE_THRESHOLD, MEMTABLE_TYPE, MERGE_MODE_KEY, SST_FORMAT_KEY,
|
||||
TWCS_FALLBACK_TO_LOCAL, TWCS_MAX_OUTPUT_FILE_SIZE, TWCS_TIME_WINDOW, TWCS_TRIGGER_FILE_NUM,
|
||||
is_mito_engine_option_key,
|
||||
};
|
||||
@@ -88,6 +90,10 @@ static VALID_DB_OPT_KEYS: Lazy<HashSet<&str>> = Lazy::new(|| {
|
||||
set.insert(TTL_KEY);
|
||||
set.insert(STORAGE_KEY);
|
||||
set.insert(MEMTABLE_TYPE);
|
||||
set.insert(MEMTABLE_BULK_MERGE_THRESHOLD);
|
||||
set.insert(MEMTABLE_BULK_ENCODE_ROW_THRESHOLD);
|
||||
set.insert(MEMTABLE_BULK_ENCODE_BYTES_THRESHOLD);
|
||||
set.insert(MEMTABLE_BULK_MAX_MERGE_GROUPS);
|
||||
set.insert(APPEND_MODE_KEY);
|
||||
set.insert(MERGE_MODE_KEY);
|
||||
set.insert(SKIP_WAL_KEY);
|
||||
@@ -482,9 +488,22 @@ mod tests {
|
||||
assert!(validate_table_option(TTL_KEY));
|
||||
assert!(validate_table_option(WRITE_BUFFER_SIZE_KEY));
|
||||
assert!(validate_table_option(STORAGE_KEY));
|
||||
assert!(validate_table_option(MEMTABLE_BULK_MERGE_THRESHOLD));
|
||||
assert!(!validate_table_option("foo"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_validate_database_option() {
|
||||
assert!(validate_database_option(MEMTABLE_TYPE));
|
||||
assert!(validate_database_option(MEMTABLE_BULK_MERGE_THRESHOLD));
|
||||
assert!(validate_database_option(MEMTABLE_BULK_ENCODE_ROW_THRESHOLD));
|
||||
assert!(validate_database_option(
|
||||
MEMTABLE_BULK_ENCODE_BYTES_THRESHOLD
|
||||
));
|
||||
assert!(validate_database_option(MEMTABLE_BULK_MAX_MERGE_GROUPS));
|
||||
assert!(!validate_database_option("foo"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_serialize_table_options() {
|
||||
let options = TableOptions {
|
||||
|
||||
@@ -1658,10 +1658,7 @@ create_on_flush = "auto"
|
||||
create_on_compaction = "auto"
|
||||
apply_on_query = "auto"
|
||||
mem_threshold_on_create = "auto"
|
||||
{vector_index_config}[region_engine.mito.memtable]
|
||||
type = "time_series"
|
||||
|
||||
[region_engine.mito.gc]
|
||||
{vector_index_config}[region_engine.mito.gc]
|
||||
enable = false
|
||||
lingering_time = "1m"
|
||||
unknown_file_lingering_time = "1h"
|
||||
|
||||
@@ -564,9 +564,8 @@ pub async fn test_repartition_metric(
|
||||
) ENGINE = metric
|
||||
WITH (
|
||||
"physical_metric_table" = "",
|
||||
"memtable.type" = "partition_tree",
|
||||
'sst_format' = '{sst_format}',
|
||||
"memtable.partition_tree.primary_key_encoding" = "{primary_key_encoding}",
|
||||
"primary_key_encoding" = "{primary_key_encoding}",
|
||||
"index.type" = "inverted",
|
||||
);
|
||||
"#
|
||||
|
||||
@@ -73,7 +73,7 @@ Affected Rows: 0
|
||||
---test more options----
|
||||
CREATE DATABASE mydb WITH (
|
||||
ttl = '1h',
|
||||
'memtable.type'='partition_tree',
|
||||
'memtable.type'='bulk',
|
||||
'append_mode'='false',
|
||||
'merge_mode'='last_non_null',
|
||||
'compaction.type' = 'twcs',
|
||||
@@ -97,7 +97,7 @@ SHOW FULL DATABASES;
|
||||
| | 'append_mode'='false' |
|
||||
| | 'compaction.twcs.time_window'='1h' |
|
||||
| | 'compaction.type'='twcs' |
|
||||
| | 'memtable.type'='partition_tree' |
|
||||
| | 'memtable.type'='bulk' |
|
||||
| | 'merge_mode'='last_non_null' |
|
||||
| | 'skip_wal'='true' |
|
||||
| | |
|
||||
@@ -110,25 +110,25 @@ Affected Rows: 0
|
||||
|
||||
SHOW CREATE TABLE test1;
|
||||
|
||||
+-------+---------------------------------------+
|
||||
| Table | Create Table |
|
||||
+-------+---------------------------------------+
|
||||
| test1 | CREATE TABLE IF NOT EXISTS "test1" ( |
|
||||
| | "host" STRING NULL, |
|
||||
| | "cpu" DOUBLE NULL, |
|
||||
| | "ts" TIMESTAMP(3) NOT NULL, |
|
||||
| | TIME INDEX ("ts") |
|
||||
| | ) |
|
||||
| | |
|
||||
| | ENGINE=mito |
|
||||
| | WITH( |
|
||||
| | append_mode = 'false', |
|
||||
| | 'memtable.type' = 'partition_tree', |
|
||||
| | merge_mode = 'last_non_null', |
|
||||
| | skip_wal = 'true', |
|
||||
| | ttl = '1h' |
|
||||
| | ) |
|
||||
+-------+---------------------------------------+
|
||||
+-------+--------------------------------------+
|
||||
| Table | Create Table |
|
||||
+-------+--------------------------------------+
|
||||
| test1 | CREATE TABLE IF NOT EXISTS "test1" ( |
|
||||
| | "host" STRING NULL, |
|
||||
| | "cpu" DOUBLE NULL, |
|
||||
| | "ts" TIMESTAMP(3) NOT NULL, |
|
||||
| | TIME INDEX ("ts") |
|
||||
| | ) |
|
||||
| | |
|
||||
| | ENGINE=mito |
|
||||
| | WITH( |
|
||||
| | append_mode = 'false', |
|
||||
| | 'memtable.type' = 'bulk', |
|
||||
| | merge_mode = 'last_non_null', |
|
||||
| | skip_wal = 'true', |
|
||||
| | ttl = '1h' |
|
||||
| | ) |
|
||||
+-------+--------------------------------------+
|
||||
|
||||
CREATE TABLE test2(host STRING, cpu DOUBLE, ts TIMESTAMP TIME INDEX) WITH (
|
||||
'append_mode'='true',
|
||||
@@ -139,25 +139,25 @@ Affected Rows: 0
|
||||
|
||||
SHOW CREATE TABLE test2;
|
||||
|
||||
+-------+---------------------------------------+
|
||||
| Table | Create Table |
|
||||
+-------+---------------------------------------+
|
||||
| test2 | CREATE TABLE IF NOT EXISTS "test2" ( |
|
||||
| | "host" STRING NULL, |
|
||||
| | "cpu" DOUBLE NULL, |
|
||||
| | "ts" TIMESTAMP(3) NOT NULL, |
|
||||
| | TIME INDEX ("ts") |
|
||||
| | ) |
|
||||
| | |
|
||||
| | ENGINE=mito |
|
||||
| | WITH( |
|
||||
| | append_mode = 'true', |
|
||||
| | 'memtable.type' = 'partition_tree', |
|
||||
| | merge_mode = '', |
|
||||
| | skip_wal = 'false', |
|
||||
| | ttl = '1h' |
|
||||
| | ) |
|
||||
+-------+---------------------------------------+
|
||||
+-------+--------------------------------------+
|
||||
| Table | Create Table |
|
||||
+-------+--------------------------------------+
|
||||
| test2 | CREATE TABLE IF NOT EXISTS "test2" ( |
|
||||
| | "host" STRING NULL, |
|
||||
| | "cpu" DOUBLE NULL, |
|
||||
| | "ts" TIMESTAMP(3) NOT NULL, |
|
||||
| | TIME INDEX ("ts") |
|
||||
| | ) |
|
||||
| | |
|
||||
| | ENGINE=mito |
|
||||
| | WITH( |
|
||||
| | append_mode = 'true', |
|
||||
| | 'memtable.type' = 'bulk', |
|
||||
| | merge_mode = '', |
|
||||
| | skip_wal = 'false', |
|
||||
| | ttl = '1h' |
|
||||
| | ) |
|
||||
+-------+--------------------------------------+
|
||||
|
||||
INSERT INTO test2 VALUES('host1', 1.0, '2023-10-01 00:00:00');
|
||||
|
||||
|
||||
@@ -19,7 +19,7 @@ DROP DATABASE mydb;
|
||||
---test more options----
|
||||
CREATE DATABASE mydb WITH (
|
||||
ttl = '1h',
|
||||
'memtable.type'='partition_tree',
|
||||
'memtable.type'='bulk',
|
||||
'append_mode'='false',
|
||||
'merge_mode'='last_non_null',
|
||||
'compaction.type' = 'twcs',
|
||||
|
||||
@@ -72,7 +72,7 @@ with(
|
||||
'index.inverted_index.ignore_column_ids'='1,2,3',
|
||||
'index.inverted_index.segment_row_count'='512',
|
||||
'wal_options'='{"wal.provider":"raft_engine"}',
|
||||
'memtable.type' = 'partition_tree',
|
||||
'memtable.type' = 'bulk',
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
@@ -62,7 +62,7 @@ with(
|
||||
'index.inverted_index.ignore_column_ids'='1,2,3',
|
||||
'index.inverted_index.segment_row_count'='512',
|
||||
'wal_options'='{"wal.provider":"raft_engine"}',
|
||||
'memtable.type' = 'partition_tree',
|
||||
'memtable.type' = 'bulk',
|
||||
);
|
||||
|
||||
drop table test_mito_options;
|
||||
|
||||
@@ -92,7 +92,7 @@ ADMIN FLUSH_TABLE('sst_case');
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
SELECT * FROM information_schema.ssts_manifest order by file_path;
|
||||
@@ -100,14 +100,14 @@ SELECT * FROM information_schema.ssts_manifest order by file_path;
|
||||
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+---------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
|
||||
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | index_version | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | num_series | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
|
||||
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+---------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+---------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+/index/<UUID>\.puffin) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
@@ -129,19 +129,19 @@ SELECT * FROM information_schema.ssts_index_meta ORDER BY meta_json;
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
SELECT * FROM information_schema.ssts_storage order by file_path;
|
||||
|
||||
+---------------------------------------------------------------------------------------------+-----------+-------------------------+---------+
|
||||
| file_path | file_size | last_modified_ms | node_id |
|
||||
+---------------------------------------------------------------------------------------------+-----------+-------------------------+---------+
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
+---------------------------------------------------------------------------------------------+-----------+-------------------------+---------+
|
||||
|
||||
INSERT INTO sst_case VALUES
|
||||
@@ -161,7 +161,7 @@ ADMIN FLUSH_TABLE('sst_case');
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
SELECT * FROM information_schema.ssts_manifest order by file_path;
|
||||
@@ -169,16 +169,16 @@ SELECT * FROM information_schema.ssts_manifest order by file_path;
|
||||
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+---------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
|
||||
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | index_version | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | num_series | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
|
||||
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+---------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>|<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>|<NUM>| <DATETIME>| <DATETIME>|<NUM>|<NUM>|<NUM>| true |
|
||||
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+---------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+------------+-------------------------+-------------------------+----------+------------------+---------+---------+
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+/index/<UUID>\.puffin) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
@@ -206,23 +206,23 @@ SELECT * FROM information_schema.ssts_index_meta ORDER BY meta_json;
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
SELECT * FROM information_schema.ssts_storage order by file_path;
|
||||
|
||||
+---------------------------------------------------------------------------------------------+-----------+-------------------------+---------+
|
||||
| file_path | file_size | last_modified_ms | node_id |
|
||||
+---------------------------------------------------------------------------------------------+-----------+-------------------------+---------+
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME> |<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| <DATETIME>|<NUM>|
|
||||
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>| <DATETIME>|<NUM>|
|
||||
+---------------------------------------------------------------------------------------------+-----------+-------------------------+---------+
|
||||
|
||||
DROP TABLE sst_case;
|
||||
|
||||
@@ -25,14 +25,14 @@ ADMIN FLUSH_TABLE('sst_case');
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
SELECT * FROM information_schema.ssts_manifest order by file_path;
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+/index/<UUID>\.puffin) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
@@ -40,7 +40,7 @@ SELECT * FROM information_schema.ssts_index_meta ORDER BY meta_json;
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
SELECT * FROM information_schema.ssts_storage order by file_path;
|
||||
|
||||
@@ -53,14 +53,14 @@ ADMIN FLUSH_TABLE('sst_case');
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
SELECT * FROM information_schema.ssts_manifest order by file_path;
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+/index/<UUID>\.puffin) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
|
||||
@@ -68,7 +68,7 @@ SELECT * FROM information_schema.ssts_index_meta ORDER BY meta_json;
|
||||
|
||||
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
|
||||
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d+)?[[:blank:]]*\|) <DATETIME> |
|
||||
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}(\.\d{3,9})?[[:blank:]]*) <DATETIME>
|
||||
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
|
||||
SELECT * FROM information_schema.ssts_storage order by file_path;
|
||||
|
||||
|
||||
@@ -116,8 +116,8 @@ CREATE TABLE phy (
|
||||
val double
|
||||
) engine = metric with (
|
||||
"physical_metric_table" = "",
|
||||
"memtable.type" = "partition_tree",
|
||||
"memtable.partition_tree.primary_key_encoding" = "sparse"
|
||||
"memtable.type" = "bulk",
|
||||
"primary_key_encoding" = "sparse"
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
@@ -44,8 +44,8 @@ CREATE TABLE phy (
|
||||
val double
|
||||
) engine = metric with (
|
||||
"physical_metric_table" = "",
|
||||
"memtable.type" = "partition_tree",
|
||||
"memtable.partition_tree.primary_key_encoding" = "sparse"
|
||||
"memtable.type" = "bulk",
|
||||
"primary_key_encoding" = "sparse"
|
||||
);
|
||||
|
||||
CREATE TABLE t1 (ts timestamp time index, val double, host string primary key) engine = metric with ("on_physical_table" = "phy");
|
||||
|
||||
Reference in New Issue
Block a user