Merge branch 'main' into dom/mutex-pushdown
commit
4c590bdb43
|
@ -176,6 +176,26 @@ macro_rules! gen_compactor_config {
|
||||||
action
|
action
|
||||||
)]
|
)]
|
||||||
pub minutes_without_new_writes_to_be_cold: u64,
|
pub minutes_without_new_writes_to_be_cold: u64,
|
||||||
|
|
||||||
|
/// When querying for partitions with data for hot compaction, how many hours to look
|
||||||
|
/// back for a first pass.
|
||||||
|
#[clap(
|
||||||
|
long = "compaction-hot-partition-hours-threshold-1",
|
||||||
|
env = "INFLUXDB_IOX_COMPACTION_HOT_PARTITION_HOURS_THRESHOLD_1",
|
||||||
|
default_value = "4",
|
||||||
|
action
|
||||||
|
)]
|
||||||
|
pub hot_compaction_hours_threshold_1: u64,
|
||||||
|
|
||||||
|
/// When querying for partitions with data for hot compaction, how many hours to look
|
||||||
|
/// back for a second pass if we found nothing in the first pass.
|
||||||
|
#[clap(
|
||||||
|
long = "compaction-hot-partition-hours-threshold-2",
|
||||||
|
env = "INFLUXDB_IOX_COMPACTION_HOT_PARTITION_HOURS_THRESHOLD_2",
|
||||||
|
default_value = "24",
|
||||||
|
action
|
||||||
|
)]
|
||||||
|
pub hot_compaction_hours_threshold_2: u64,
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
@ -204,6 +224,8 @@ impl CompactorOnceConfig {
|
||||||
.min_num_rows_allocated_per_record_batch_to_datafusion_plan,
|
.min_num_rows_allocated_per_record_batch_to_datafusion_plan,
|
||||||
max_num_compacting_files: self.max_num_compacting_files,
|
max_num_compacting_files: self.max_num_compacting_files,
|
||||||
minutes_without_new_writes_to_be_cold: self.minutes_without_new_writes_to_be_cold,
|
minutes_without_new_writes_to_be_cold: self.minutes_without_new_writes_to_be_cold,
|
||||||
|
hot_compaction_hours_threshold_1: self.hot_compaction_hours_threshold_1,
|
||||||
|
hot_compaction_hours_threshold_2: self.hot_compaction_hours_threshold_2,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -112,6 +112,9 @@ mod tests {
|
||||||
use parquet_file::storage::StorageId;
|
use parquet_file::storage::StorageId;
|
||||||
use std::collections::HashMap;
|
use std::collections::HashMap;
|
||||||
|
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1: u64 = 4;
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2: u64 = 24;
|
||||||
|
|
||||||
#[tokio::test]
|
#[tokio::test]
|
||||||
async fn test_compact_remaining_level_0_files_many_files() {
|
async fn test_compact_remaining_level_0_files_many_files() {
|
||||||
test_helpers::maybe_start_logging();
|
test_helpers::maybe_start_logging();
|
||||||
|
@ -701,6 +704,8 @@ mod tests {
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 1,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 1,
|
||||||
max_num_compacting_files: 20,
|
max_num_compacting_files: 20,
|
||||||
minutes_without_new_writes_to_be_cold: 10,
|
minutes_without_new_writes_to_be_cold: 10,
|
||||||
|
hot_compaction_hours_threshold_1: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
hot_compaction_hours_threshold_2: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -258,7 +258,8 @@ impl Compactor {
|
||||||
/// Return a list of partitions that:
|
/// Return a list of partitions that:
|
||||||
///
|
///
|
||||||
/// - Have not received any writes in 8 hours (determined by all level 0 and level 1 parquet
|
/// - Have not received any writes in 8 hours (determined by all level 0 and level 1 parquet
|
||||||
/// files having a created_at time older than 8 hours ago)
|
/// files having a created_at time older than 8 hours ago). Note that 8 is the default but
|
||||||
|
/// it's configurable
|
||||||
/// - Have some level 0 or level 1 parquet files that need to be upgraded or compacted
|
/// - Have some level 0 or level 1 parquet files that need to be upgraded or compacted
|
||||||
/// - Sorted by the number of level 0 files + number of level 1 files descending
|
/// - Sorted by the number of level 0 files + number of level 1 files descending
|
||||||
pub async fn cold_partitions_to_compact(
|
pub async fn cold_partitions_to_compact(
|
||||||
|
@ -559,6 +560,9 @@ pub mod tests {
|
||||||
use parquet_file::storage::StorageId;
|
use parquet_file::storage::StorageId;
|
||||||
use uuid::Uuid;
|
use uuid::Uuid;
|
||||||
|
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1: u64 = 4;
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2: u64 = 24;
|
||||||
|
|
||||||
impl PartitionCompactionCandidateWithInfo {
|
impl PartitionCompactionCandidateWithInfo {
|
||||||
pub(crate) async fn from_test_partition(test_partition: &TestPartition) -> Self {
|
pub(crate) async fn from_test_partition(test_partition: &TestPartition) -> Self {
|
||||||
Self {
|
Self {
|
||||||
|
@ -690,6 +694,8 @@ pub mod tests {
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
||||||
max_num_compacting_files: 20,
|
max_num_compacting_files: 20,
|
||||||
minutes_without_new_writes_to_be_cold: 10,
|
minutes_without_new_writes_to_be_cold: 10,
|
||||||
|
hot_compaction_hours_threshold_1: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
hot_compaction_hours_threshold_2: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -159,6 +159,14 @@ pub struct CompactorConfig {
|
||||||
|
|
||||||
/// Minutes without any new data before a partition is considered cold
|
/// Minutes without any new data before a partition is considered cold
|
||||||
pub minutes_without_new_writes_to_be_cold: u64,
|
pub minutes_without_new_writes_to_be_cold: u64,
|
||||||
|
|
||||||
|
/// When querying for partitions with data for hot compaction, how many hours to look
|
||||||
|
/// back for a first pass.
|
||||||
|
pub hot_compaction_hours_threshold_1: u64,
|
||||||
|
|
||||||
|
/// When querying for partitions with data for hot compaction, how many hours to look
|
||||||
|
/// back for a second pass if we found nothing in the first pass.
|
||||||
|
pub hot_compaction_hours_threshold_2: u64,
|
||||||
}
|
}
|
||||||
|
|
||||||
/// How long to pause before checking for more work again if there was
|
/// How long to pause before checking for more work again if there was
|
||||||
|
|
|
@ -60,9 +60,10 @@ pub async fn compact(compactor: Arc<Compactor>) -> usize {
|
||||||
|
|
||||||
/// Return a list of the most recent highest ingested throughput partitions.
|
/// Return a list of the most recent highest ingested throughput partitions.
|
||||||
/// The highest throughput partitions are prioritized as follows:
|
/// The highest throughput partitions are prioritized as follows:
|
||||||
/// 1. If there are partitions with new ingested files within the last 4 hours, pick them.
|
/// 1. If there are partitions with new ingested files within the last 4 hours (the default, but
|
||||||
|
/// configurable), pick them.
|
||||||
/// 2. If no new ingested files in the last 4 hours, will look for partitions with new writes
|
/// 2. If no new ingested files in the last 4 hours, will look for partitions with new writes
|
||||||
/// within the last 24 hours.
|
/// within the last 24 hours (the default, but configurable).
|
||||||
/// 3. If there are no ingested files within the last 24 hours, will look for partitions
|
/// 3. If there are no ingested files within the last 24 hours, will look for partitions
|
||||||
/// with any new ingested files in the past.
|
/// with any new ingested files in the past.
|
||||||
///
|
///
|
||||||
|
@ -84,7 +85,11 @@ pub(crate) async fn hot_partitions_to_compact(
|
||||||
|
|
||||||
// Get the most recent highest ingested throughput partitions within the last 4 hours. If not,
|
// Get the most recent highest ingested throughput partitions within the last 4 hours. If not,
|
||||||
// increase to 24 hours.
|
// increase to 24 hours.
|
||||||
let query_times = query_times(compactor.time_provider());
|
let query_times = query_times(
|
||||||
|
compactor.time_provider(),
|
||||||
|
compactor.config.hot_compaction_hours_threshold_1,
|
||||||
|
compactor.config.hot_compaction_hours_threshold_2,
|
||||||
|
);
|
||||||
|
|
||||||
for &shard_id in &compactor.shards {
|
for &shard_id in &compactor.shards {
|
||||||
let mut partitions = hot_partitions_for_shard(
|
let mut partitions = hot_partitions_for_shard(
|
||||||
|
@ -191,8 +196,12 @@ async fn hot_partitions_for_shard(
|
||||||
Ok(Vec::new())
|
Ok(Vec::new())
|
||||||
}
|
}
|
||||||
|
|
||||||
fn query_times(time_provider: Arc<dyn TimeProvider>) -> Vec<(u64, Timestamp)> {
|
fn query_times(
|
||||||
[4, 24]
|
time_provider: Arc<dyn TimeProvider>,
|
||||||
|
hours_threshold_1: u64,
|
||||||
|
hours_threshold_2: u64,
|
||||||
|
) -> Vec<(u64, Timestamp)> {
|
||||||
|
[hours_threshold_1, hours_threshold_2]
|
||||||
.iter()
|
.iter()
|
||||||
.map(|&num_hours| {
|
.map(|&num_hours| {
|
||||||
(
|
(
|
||||||
|
@ -213,6 +222,9 @@ mod tests {
|
||||||
use parquet_file::storage::{ParquetStorage, StorageId};
|
use parquet_file::storage::{ParquetStorage, StorageId};
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1: u64 = 4;
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2: u64 = 24;
|
||||||
|
|
||||||
struct TestSetup {
|
struct TestSetup {
|
||||||
catalog: Arc<TestCatalog>,
|
catalog: Arc<TestCatalog>,
|
||||||
shard1: Arc<TestShard>,
|
shard1: Arc<TestShard>,
|
||||||
|
@ -247,7 +259,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -271,7 +287,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -298,7 +318,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -326,7 +350,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -354,7 +382,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -382,7 +414,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -410,7 +446,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
1,
|
1,
|
||||||
)
|
)
|
||||||
|
@ -447,7 +487,11 @@ mod tests {
|
||||||
let candidates = hot_partitions_for_shard(
|
let candidates = hot_partitions_for_shard(
|
||||||
Arc::clone(&catalog.catalog),
|
Arc::clone(&catalog.catalog),
|
||||||
shard1.shard.id,
|
shard1.shard.id,
|
||||||
&query_times(catalog.time_provider()),
|
&query_times(
|
||||||
|
catalog.time_provider(),
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
|
),
|
||||||
1,
|
1,
|
||||||
// Even if we ask for 2 partitions per shard, we'll only get the one partition with
|
// Even if we ask for 2 partitions per shard, we'll only get the one partition with
|
||||||
// writes within 4 hours
|
// writes within 4 hours
|
||||||
|
@ -495,6 +539,8 @@ mod tests {
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
||||||
max_num_compacting_files: 20,
|
max_num_compacting_files: 20,
|
||||||
minutes_without_new_writes_to_be_cold: 10,
|
minutes_without_new_writes_to_be_cold: 10,
|
||||||
|
hot_compaction_hours_threshold_1: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
hot_compaction_hours_threshold_2: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
};
|
};
|
||||||
let compactor = Arc::new(Compactor::new(
|
let compactor = Arc::new(Compactor::new(
|
||||||
vec![shard1.shard.id, shard2.shard.id],
|
vec![shard1.shard.id, shard2.shard.id],
|
||||||
|
|
|
@ -448,6 +448,9 @@ pub mod tests {
|
||||||
sync::{Arc, Mutex},
|
sync::{Arc, Mutex},
|
||||||
};
|
};
|
||||||
|
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1: u64 = 4;
|
||||||
|
const DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2: u64 = 24;
|
||||||
|
|
||||||
// In tests that are verifying successful compaction not affected by the memory budget, this
|
// In tests that are verifying successful compaction not affected by the memory budget, this
|
||||||
// converts a `parquet_file_filtering::FilteredFiles` that has a `filter_result` of
|
// converts a `parquet_file_filtering::FilteredFiles` that has a `filter_result` of
|
||||||
// `parquet_file_filtering::FilterResult::Proceed` into a `ReadyToCompact` and panics if it
|
// `parquet_file_filtering::FilterResult::Proceed` into a `ReadyToCompact` and panics if it
|
||||||
|
@ -562,6 +565,8 @@ pub mod tests {
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 2,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 2,
|
||||||
max_num_compacting_files: 20,
|
max_num_compacting_files: 20,
|
||||||
minutes_without_new_writes_to_be_cold: 10,
|
minutes_without_new_writes_to_be_cold: 10,
|
||||||
|
hot_compaction_hours_threshold_1: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
hot_compaction_hours_threshold_2: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -941,6 +946,8 @@ pub mod tests {
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
||||||
max_num_compacting_files: 20,
|
max_num_compacting_files: 20,
|
||||||
minutes_without_new_writes_to_be_cold: 10,
|
minutes_without_new_writes_to_be_cold: 10,
|
||||||
|
hot_compaction_hours_threshold_1: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_1,
|
||||||
|
hot_compaction_hours_threshold_2: DEFAULT_HOT_COMPACTION_HOURS_THRESHOLD_2,
|
||||||
};
|
};
|
||||||
|
|
||||||
let metrics = Arc::new(metric::Registry::new());
|
let metrics = Arc::new(metric::Registry::new());
|
||||||
|
|
|
@ -427,6 +427,8 @@ impl Config {
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan: 100,
|
||||||
max_num_compacting_files: 20,
|
max_num_compacting_files: 20,
|
||||||
minutes_without_new_writes_to_be_cold: 10,
|
minutes_without_new_writes_to_be_cold: 10,
|
||||||
|
hot_compaction_hours_threshold_1: 4,
|
||||||
|
hot_compaction_hours_threshold_2: 24,
|
||||||
};
|
};
|
||||||
|
|
||||||
let querier_config = QuerierConfig {
|
let querier_config = QuerierConfig {
|
||||||
|
|
|
@ -211,6 +211,8 @@ pub async fn build_compactor_from_config(
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan,
|
||||||
max_num_compacting_files,
|
max_num_compacting_files,
|
||||||
minutes_without_new_writes_to_be_cold,
|
minutes_without_new_writes_to_be_cold,
|
||||||
|
hot_compaction_hours_threshold_1,
|
||||||
|
hot_compaction_hours_threshold_2,
|
||||||
..
|
..
|
||||||
} = compactor_config;
|
} = compactor_config;
|
||||||
|
|
||||||
|
@ -225,6 +227,8 @@ pub async fn build_compactor_from_config(
|
||||||
min_num_rows_allocated_per_record_batch_to_datafusion_plan,
|
min_num_rows_allocated_per_record_batch_to_datafusion_plan,
|
||||||
max_num_compacting_files,
|
max_num_compacting_files,
|
||||||
minutes_without_new_writes_to_be_cold,
|
minutes_without_new_writes_to_be_cold,
|
||||||
|
hot_compaction_hours_threshold_1,
|
||||||
|
hot_compaction_hours_threshold_2,
|
||||||
};
|
};
|
||||||
|
|
||||||
Ok(compactor::compact::Compactor::new(
|
Ok(compactor::compact::Compactor::new(
|
||||||
|
|
Loading…
Reference in New Issue