Skip to content

Commit

Permalink
fix!: fix regression caused by unbalanced partitions and splitting ra…
Browse files Browse the repository at this point in the history
…nges (#5090)

* feat: assign partition ranges by rows

* feat: balance partition rows

* feat: get uppoer bound for part nums

* feat: only split in non-compaction seq scan

* fix: parallel scan on multiple sources

* fix: can split check

* feat: scanner prepare by request

* feat: remove scan_parallelism

* docs: upate docs

* chore: update comment

* style: fix clippy

* feat: skip merge and dedup if there is only one source

* chore: Revert "feat: skip merge and dedup if there is only one source"

Since memtable won't do dedup jobs

This reverts commit 2fc7a54.

* test: avoid compaction in sqlness window sort test

* chore: do not create semaphore if num partitions is enough

* chore: more assertions

* chore: fix typo

* fix: compaction flag not set

* chore: address review comments
  • Loading branch information
evenyag authored Dec 9, 2024
1 parent 1b642ea commit 2fcb95f
Show file tree
Hide file tree
Showing 19 changed files with 554 additions and 303 deletions.
2 changes: 0 additions & 2 deletions config/config.md
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,6 @@
| `region_engine.mito.experimental_write_cache_size` | String | `1GiB` | Capacity for write cache. If your disk space is sufficient, it is recommended to set it larger. |
| `region_engine.mito.experimental_write_cache_ttl` | String | Unset | TTL for write cache. |
| `region_engine.mito.sst_write_buffer_size` | String | `8MB` | Buffer size for SST writing. |
| `region_engine.mito.scan_parallelism` | Integer | `0` | Parallelism to scan a region (default: 1/4 of cpu cores).<br/>- `0`: using the default value (1/4 of cpu cores).<br/>- `1`: scan in current thread.<br/>- `n`: scan in parallelism n. |
| `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. |
| `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. |
| `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.<br/>To align with the old behavior, the default value is 0 (no restrictions). |
Expand Down Expand Up @@ -464,7 +463,6 @@
| `region_engine.mito.experimental_write_cache_size` | String | `1GiB` | Capacity for write cache. If your disk space is sufficient, it is recommended to set it larger. |
| `region_engine.mito.experimental_write_cache_ttl` | String | Unset | TTL for write cache. |
| `region_engine.mito.sst_write_buffer_size` | String | `8MB` | Buffer size for SST writing. |
| `region_engine.mito.scan_parallelism` | Integer | `0` | Parallelism to scan a region (default: 1/4 of cpu cores).<br/>- `0`: using the default value (1/4 of cpu cores).<br/>- `1`: scan in current thread.<br/>- `n`: scan in parallelism n. |
| `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. |
| `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. |
| `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.<br/>To align with the old behavior, the default value is 0 (no restrictions). |
Expand Down
6 changes: 0 additions & 6 deletions config/datanode.example.toml
Original file line number Diff line number Diff line change
Expand Up @@ -492,12 +492,6 @@ experimental_write_cache_ttl = "8h"
## Buffer size for SST writing.
sst_write_buffer_size = "8MB"

## Parallelism to scan a region (default: 1/4 of cpu cores).
## - `0`: using the default value (1/4 of cpu cores).
## - `1`: scan in current thread.
## - `n`: scan in parallelism n.
scan_parallelism = 0

## Capacity of the channel to send data from parallel scan tasks to the main task.
parallel_scan_channel_size = 32

Expand Down
6 changes: 0 additions & 6 deletions config/standalone.example.toml
Original file line number Diff line number Diff line change
Expand Up @@ -530,12 +530,6 @@ experimental_write_cache_ttl = "8h"
## Buffer size for SST writing.
sst_write_buffer_size = "8MB"

## Parallelism to scan a region (default: 1/4 of cpu cores).
## - `0`: using the default value (1/4 of cpu cores).
## - `1`: scan in current thread.
## - `n`: scan in parallelism n.
scan_parallelism = 0

## Capacity of the channel to send data from parallel scan tasks to the main task.
parallel_scan_channel_size = 32

Expand Down
2 changes: 0 additions & 2 deletions src/cmd/tests/load_config_test.rs
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@ fn test_load_datanode_example_config() {
region_engine: vec![
RegionEngineConfig::Mito(MitoConfig {
auto_flush_interval: Duration::from_secs(3600),
scan_parallelism: 0,
experimental_write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)),
..Default::default()
}),
Expand Down Expand Up @@ -205,7 +204,6 @@ fn test_load_standalone_example_config() {
RegionEngineConfig::Mito(MitoConfig {
auto_flush_interval: Duration::from_secs(3600),
experimental_write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)),
scan_parallelism: 0,
..Default::default()
}),
RegionEngineConfig::File(EngineConfig {}),
Expand Down
5 changes: 2 additions & 3 deletions src/mito2/src/compaction.rs
Original file line number Diff line number Diff line change
Expand Up @@ -597,9 +597,8 @@ impl<'a> CompactionSstReaderBuilder<'a> {
scan_input.with_predicate(time_range_to_predicate(time_range, &self.metadata)?);
}

SeqScan::new(scan_input)
.with_compaction()
.build_reader()
SeqScan::new(scan_input, true)
.build_reader_for_compaction()
.await
}
}
Expand Down
13 changes: 1 addition & 12 deletions src/mito2/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ use crate::sst::DEFAULT_WRITE_BUFFER_SIZE;

const MULTIPART_UPLOAD_MINIMUM_SIZE: ReadableSize = ReadableSize::mb(5);
/// Default channel size for parallel scan task.
const DEFAULT_SCAN_CHANNEL_SIZE: usize = 32;
pub(crate) const DEFAULT_SCAN_CHANNEL_SIZE: usize = 32;

// Use `1/GLOBAL_WRITE_BUFFER_SIZE_FACTOR` of OS memory as global write buffer size in default mode
const GLOBAL_WRITE_BUFFER_SIZE_FACTOR: u64 = 8;
Expand Down Expand Up @@ -107,11 +107,6 @@ pub struct MitoConfig {
// Other configs:
/// Buffer size for SST writing.
pub sst_write_buffer_size: ReadableSize,
/// Parallelism to scan a region (default: 1/4 of cpu cores).
/// - 0: using the default value (1/4 of cpu cores).
/// - 1: scan in current thread.
/// - n: scan in parallelism n.
pub scan_parallelism: usize,
/// Capacity of the channel to send data from parallel scan tasks to the main task (default 32).
pub parallel_scan_channel_size: usize,
/// Whether to allow stale entries read during replay.
Expand Down Expand Up @@ -156,7 +151,6 @@ impl Default for MitoConfig {
experimental_write_cache_size: ReadableSize::gb(1),
experimental_write_cache_ttl: None,
sst_write_buffer_size: DEFAULT_WRITE_BUFFER_SIZE,
scan_parallelism: divide_num_cpus(4),
parallel_scan_channel_size: DEFAULT_SCAN_CHANNEL_SIZE,
allow_stale_entries: false,
index: IndexConfig::default(),
Expand Down Expand Up @@ -229,11 +223,6 @@ impl MitoConfig {
);
}

// Use default value if `scan_parallelism` is 0.
if self.scan_parallelism == 0 {
self.scan_parallelism = divide_num_cpus(4);
}

if self.parallel_scan_channel_size < 1 {
self.parallel_scan_channel_size = DEFAULT_SCAN_CHANNEL_SIZE;
warn!(
Expand Down
26 changes: 7 additions & 19 deletions src/mito2/src/engine.rs
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ use crate::error::{
};
use crate::manifest::action::RegionEdit;
use crate::metrics::HANDLE_REQUEST_ELAPSED;
use crate::read::scan_region::{ScanParallelism, ScanRegion, Scanner};
use crate::read::scan_region::{ScanRegion, Scanner};
use crate::request::{RegionEditRequest, WorkerRequest};
use crate::wal::entry_distributor::{
build_wal_entry_distributor_and_receivers, DEFAULT_ENTRY_RECEIVER_BUFFER_SIZE,
Expand Down Expand Up @@ -171,19 +171,9 @@ impl MitoEngine {
self.scan_region(region_id, request)?.scanner()
}

/// Returns a region scanner to scan the region for `request`.
fn region_scanner(
&self,
region_id: RegionId,
request: ScanRequest,
) -> Result<RegionScannerRef> {
let scanner = self.scanner(region_id, request)?;
scanner.region_scanner()
}

/// Scans a region.
fn scan_region(&self, region_id: RegionId, request: ScanRequest) -> Result<ScanRegion> {
self.inner.handle_query(region_id, request)
self.inner.scan_region(region_id, request)
}

/// Edit region's metadata by [RegionEdit] directly. Use with care.
Expand Down Expand Up @@ -423,7 +413,7 @@ impl EngineInner {
}

/// Handles the scan `request` and returns a [ScanRegion].
fn handle_query(&self, region_id: RegionId, request: ScanRequest) -> Result<ScanRegion> {
fn scan_region(&self, region_id: RegionId, request: ScanRequest) -> Result<ScanRegion> {
let query_start = Instant::now();
// Reading a region doesn't need to go through the region worker thread.
let region = self
Expand All @@ -433,14 +423,10 @@ impl EngineInner {
let version = region.version();
// Get cache.
let cache_manager = self.workers.cache_manager();
let scan_parallelism = ScanParallelism {
parallelism: self.config.scan_parallelism,
channel_size: self.config.parallel_scan_channel_size,
};

let scan_region =
ScanRegion::new(version, region.access_layer.clone(), request, cache_manager)
.with_parallelism(scan_parallelism)
.with_parallel_scan_channel_size(self.config.parallel_scan_channel_size)
.with_ignore_inverted_index(self.config.inverted_index.apply_on_query.disabled())
.with_ignore_fulltext_index(self.config.fulltext_index.apply_on_query.disabled())
.with_start_time(query_start);
Expand Down Expand Up @@ -538,7 +524,9 @@ impl RegionEngine for MitoEngine {
region_id: RegionId,
request: ScanRequest,
) -> Result<RegionScannerRef, BoxedError> {
self.region_scanner(region_id, request)
self.scan_region(region_id, request)
.map_err(BoxedError::new)?
.region_scanner()
.map_err(BoxedError::new)
}

Expand Down
7 changes: 3 additions & 4 deletions src/mito2/src/engine/append_mode_test.rs
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,6 @@ async fn test_append_mode_compaction() {
let mut env = TestEnv::new();
let engine = env
.create_engine(MitoConfig {
scan_parallelism: 2,
..Default::default()
})
.await;
Expand Down Expand Up @@ -176,19 +175,19 @@ async fn test_append_mode_compaction() {
| b | 1.0 | 1970-01-01T00:00:01 |
+-------+---------+---------------------+";
// Scans in parallel.
let scanner = engine.scanner(region_id, ScanRequest::default()).unwrap();
let mut scanner = engine.scanner(region_id, ScanRequest::default()).unwrap();
assert_eq!(2, scanner.num_files());
assert_eq!(1, scanner.num_memtables());
scanner.set_target_partitions(2);
let stream = scanner.scan().await.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
assert_eq!(expected, sort_batches_and_print(&batches, &["tag_0", "ts"]));

// Reopens engine with parallelism 1.
// Reopens engine.
let engine = env
.reopen_engine(
engine,
MitoConfig {
scan_parallelism: 1,
..Default::default()
},
)
Expand Down
7 changes: 3 additions & 4 deletions src/mito2/src/engine/merge_mode_test.rs
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,6 @@ async fn test_merge_mode_compaction() {
let mut env = TestEnv::new();
let engine = env
.create_engine(MitoConfig {
scan_parallelism: 2,
..Default::default()
})
.await;
Expand Down Expand Up @@ -190,19 +189,19 @@ async fn test_merge_mode_compaction() {
| a | | 13.0 | 1970-01-01T00:00:03 |
+-------+---------+---------+---------------------+";
// Scans in parallel.
let scanner = engine.scanner(region_id, ScanRequest::default()).unwrap();
let mut scanner = engine.scanner(region_id, ScanRequest::default()).unwrap();
assert_eq!(1, scanner.num_files());
assert_eq!(1, scanner.num_memtables());
scanner.set_target_partitions(2);
let stream = scanner.scan().await.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
assert_eq!(expected, sort_batches_and_print(&batches, &["tag_0", "ts"]));

// Reopens engine with parallelism 1.
// Reopens engine.
let engine = env
.reopen_engine(
engine,
MitoConfig {
scan_parallelism: 1,
..Default::default()
},
)
Expand Down
5 changes: 3 additions & 2 deletions src/mito2/src/engine/parallel_test.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ async fn scan_in_parallel(
) {
let engine = env
.open_engine(MitoConfig {
scan_parallelism: parallelism,
parallel_scan_channel_size: channel_size,
..Default::default()
})
Expand All @@ -57,7 +56,9 @@ async fn scan_in_parallel(
.unwrap();

let request = ScanRequest::default();
let stream = engine.scan_to_stream(region_id, request).await.unwrap();
let mut scanner = engine.scanner(region_id, request).unwrap();
scanner.set_target_partitions(parallelism);
let stream = scanner.scan().await.unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
let expected = "\
+-------+---------+---------------------+
Expand Down
Loading

0 comments on commit 2fcb95f

Please sign in to comment.