From ca9759596e7d446f20e6bbdc46b017e5ca76144c Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Wed, 19 Jun 2024 15:09:50 -0400 Subject: [PATCH] Validate documents during persist (#5046) --- quickwit/Cargo.lock | 1 + quickwit/quickwit-codegen/example/Cargo.toml | 2 +- quickwit/quickwit-common/src/thread_pool.rs | 8 +- .../quickwit-config/src/index_template/mod.rs | 38 +- .../src/control_plane.rs | 1 - .../src/ingest/ingest_controller.rs | 270 +++++--- .../src/actors/doc_processor.rs | 2 +- quickwit/quickwit-ingest/Cargo.toml | 1 + .../src/ingest_v2/broadcast.rs | 2 + .../src/ingest_v2/doc_mapper.rs | 243 +++++++ .../quickwit-ingest/src/ingest_v2/idle.rs | 2 + .../quickwit-ingest/src/ingest_v2/ingester.rs | 644 ++++++++++++++---- quickwit/quickwit-ingest/src/ingest_v2/mod.rs | 78 ++- .../quickwit-ingest/src/ingest_v2/models.rs | 25 +- .../src/ingest_v2/mrecordlog_utils.rs | 8 +- .../src/ingest_v2/replication.rs | 8 +- .../quickwit-ingest/src/ingest_v2/router.rs | 45 +- .../src/ingest_v2/routing_table.rs | 2 +- .../quickwit-ingest/src/ingest_v2/state.rs | 23 +- .../src/ingest_v2/workbench.rs | 12 + .../quickwit-ingest/src/mrecordlog_async.rs | 24 +- quickwit/quickwit-metastore/src/lib.rs | 4 +- quickwit/quickwit-proto/build.rs | 3 +- .../protos/quickwit/common.proto | 10 + .../protos/quickwit/ingest.proto | 13 + .../protos/quickwit/ingester.proto | 3 + .../protos/quickwit/metastore.proto | 3 +- .../protos/quickwit/router.proto | 2 + .../src/codegen/quickwit/quickwit.common.rs | 6 +- .../quickwit/quickwit.ingest.ingester.rs | 6 + .../quickwit/quickwit.ingest.router.rs | 4 + .../src/codegen/quickwit/quickwit.ingest.rs | 44 ++ .../codegen/quickwit/quickwit.metastore.rs | 5 +- quickwit/quickwit-proto/src/getters.rs | 26 + quickwit/quickwit-proto/src/ingest/mod.rs | 42 +- .../src/types/doc_mapping_uid.rs | 2 +- quickwit/quickwit-proto/src/types/doc_uid.rs | 197 ++++++ .../quickwit-proto/src/types/index_uid.rs | 2 +- quickwit/quickwit-proto/src/types/mod.rs | 5 + .../quickwit-proto/src/types/pipeline_uid.rs | 14 +- .../src/delete_task_api/handler.rs | 2 +- .../src/elasticsearch_api/bulk_v2.rs | 371 +++++++--- .../src/elasticsearch_api/model/error.rs | 23 +- .../src/elasticsearch_api/model/mod.rs | 2 +- .../src/ingest_api/rest_handler.rs | 5 +- .../bulk/0001-happy-path.yaml | 8 +- .../bulk/0004-put-request.yaml | 9 +- ...4-validation-failed-no-requests-added.yaml | 8 - .../bulk/0005-document-parsing-exception.yaml | 16 + .../bulk/_setup.elasticsearch.yaml | 4 + .../bulk/_setup.quickwit.yaml | 2 + .../bulk/_teardown.elasticsearch.yaml | 3 - 52 files changed, 1853 insertions(+), 430 deletions(-) create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs create mode 100644 quickwit/quickwit-proto/src/types/doc_uid.rs delete mode 100644 quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-validation-failed-no-requests-added.yaml create mode 100644 quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0005-document-parsing-exception.yaml diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 6b10d53ae4b..13278547b53 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -6049,6 +6049,7 @@ dependencies = [ "quickwit-codegen", "quickwit-common", "quickwit-config", + "quickwit-doc-mapper", "quickwit-proto", "rand 0.8.5", "rand_distr", diff --git a/quickwit/quickwit-codegen/example/Cargo.toml b/quickwit/quickwit-codegen/example/Cargo.toml index 8a178d2c957..06c7f138c87 100644 --- a/quickwit/quickwit-codegen/example/Cargo.toml +++ b/quickwit/quickwit-codegen/example/Cargo.toml @@ -27,7 +27,7 @@ utoipa = { workspace = true } quickwit-actors = { workspace = true } quickwit-common = { workspace = true } -quickwit-proto ={ workspace = true } +quickwit-proto = { workspace = true } [dev-dependencies] mockall = { workspace = true } diff --git a/quickwit/quickwit-common/src/thread_pool.rs b/quickwit/quickwit-common/src/thread_pool.rs index 03d1179e826..229007e20d4 100644 --- a/quickwit/quickwit-common/src/thread_pool.rs +++ b/quickwit/quickwit-common/src/thread_pool.rs @@ -83,7 +83,7 @@ impl ThreadPool { /// but is not running yet "cancellable". pub fn run_cpu_intensive( &self, - cpu_heavy_task: F, + cpu_intensive_fn: F, ) -> impl Future> where F: FnOnce() -> R + Send + 'static, @@ -103,7 +103,7 @@ impl ThreadPool { let _guard = span.enter(); let mut ongoing_task_guard = GaugeGuard::from_gauge(&ongoing_tasks); ongoing_task_guard.add(1i64); - let result = cpu_heavy_task(); + let result = cpu_intensive_fn(); let _ = tx.send(result); }); rx.map_err(|_| Panicked) @@ -118,7 +118,7 @@ impl ThreadPool { /// /// Disclaimer: The function will no be executed if the Future is dropped. #[must_use = "run_cpu_intensive will not run if the future it returns is dropped"] -pub fn run_cpu_intensive(cpu_heavy_task: F) -> impl Future> +pub fn run_cpu_intensive(cpu_intensive_fn: F) -> impl Future> where F: FnOnce() -> R + Send + 'static, R: Send + 'static, @@ -129,7 +129,7 @@ where let num_threads: usize = (crate::num_cpus() / 3).max(2); ThreadPool::new("small_tasks", Some(num_threads)) }) - .run_cpu_intensive(cpu_heavy_task) + .run_cpu_intensive(cpu_intensive_fn) } #[derive(Clone, Copy, Debug, Eq, PartialEq)] diff --git a/quickwit/quickwit-config/src/index_template/mod.rs b/quickwit/quickwit-config/src/index_template/mod.rs index ea57a8907c0..b1c3b0d36c1 100644 --- a/quickwit/quickwit-config/src/index_template/mod.rs +++ b/quickwit/quickwit-config/src/index_template/mod.rs @@ -21,7 +21,7 @@ mod serialize; use anyhow::ensure; use quickwit_common::uri::Uri; -use quickwit_proto::types::IndexId; +use quickwit_proto::types::{DocMappingUid, IndexId}; use serde::{Deserialize, Serialize}; pub use serialize::{IndexTemplateV0_8, VersionedIndexTemplate}; @@ -68,10 +68,14 @@ impl IndexTemplate { .unwrap_or(default_index_root_uri) .join(&index_id)?; + // Ensure that the doc mapping UID is truly unique per index. + let mut doc_mapping = self.doc_mapping.clone(); + doc_mapping.doc_mapping_uid = DocMappingUid::random(); + let index_config = IndexConfig { index_id, index_uri, - doc_mapping: self.doc_mapping.clone(), + doc_mapping, indexing_settings: self.indexing_settings.clone(), search_settings: self.search_settings.clone(), retention_policy_opt: self.retention_policy_opt.clone(), @@ -235,33 +239,37 @@ mod tests { }); let default_index_root_uri = Uri::for_test("s3://test-bucket/indexes"); - let index_config = index_template - .apply_template("test-index".to_string(), &default_index_root_uri) + let index_config_foo = index_template + .apply_template("test-index-foo".to_string(), &default_index_root_uri) .unwrap(); - assert_eq!(index_config.index_id, "test-index"); - assert_eq!(index_config.index_uri, "ram:///indexes/test-index"); + assert_eq!(index_config_foo.index_id, "test-index-foo"); + assert_eq!(index_config_foo.index_uri, "ram:///indexes/test-index-foo"); - assert_eq!(index_config.doc_mapping.timestamp_field.unwrap(), "ts"); - assert_eq!(index_config.indexing_settings.commit_timeout_secs, 42); + assert_eq!(index_config_foo.doc_mapping.timestamp_field.unwrap(), "ts"); + assert_eq!(index_config_foo.indexing_settings.commit_timeout_secs, 42); assert_eq!( - index_config.search_settings.default_search_fields, + index_config_foo.search_settings.default_search_fields, ["message"] ); - let retention_policy = index_config.retention_policy_opt.unwrap(); + let retention_policy = index_config_foo.retention_policy_opt.unwrap(); assert_eq!(retention_policy.retention_period, "42 days"); assert_eq!(retention_policy.evaluation_schedule, "hourly"); index_template.index_root_uri = None; - let index_config = index_template - .apply_template("test-index".to_string(), &default_index_root_uri) + let index_config_bar = index_template + .apply_template("test-index-bar".to_string(), &default_index_root_uri) .unwrap(); - assert_eq!(index_config.index_id, "test-index"); + assert_eq!(index_config_bar.index_id, "test-index-bar"); assert_eq!( - index_config.index_uri, - "s3://test-bucket/indexes/test-index" + index_config_bar.index_uri, + "s3://test-bucket/indexes/test-index-bar" + ); + assert_ne!( + index_config_foo.doc_mapping.doc_mapping_uid, + index_config_bar.doc_mapping.doc_mapping_uid ); } diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 73cc428139a..ea16e4ef796 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -2401,7 +2401,6 @@ mod tests { control_plane_mailbox.ask(callback).await.unwrap(); let control_plane_debug_info = control_plane_mailbox.ask(GetDebugInfo).await.unwrap(); - println!("{:?}", control_plane_debug_info); let shard = &control_plane_debug_info["shard_table"]["test-index:00000000000000000000000000"][0]; assert_eq!(shard["shard_id"], "00000000000000000000"); diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 5a229f89130..9bc8898bf67 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -45,7 +45,7 @@ use quickwit_proto::ingest::{ Shard, ShardIdPosition, ShardIdPositions, ShardIds, ShardPKey, ShardState, }; use quickwit_proto::metastore::{ - MetastoreResult, MetastoreService, MetastoreServiceClient, OpenShardSubrequest, + serde_utils, MetastoreResult, MetastoreService, MetastoreServiceClient, OpenShardSubrequest, OpenShardsRequest, OpenShardsResponse, }; use quickwit_proto::types::{IndexUid, NodeId, Position, ShardId, SourceUid}; @@ -663,13 +663,13 @@ impl IngestController { let num_shards: usize = source_uids.values().sum(); if num_shards == 0 { - return Ok(HashMap::default()); + return Ok(HashMap::new()); } // TODO unavailable leaders let Some(leader_follower_pairs) = self.allocate_shards(num_shards, unavailable_leaders, model) else { - return Ok(HashMap::default()); + return Ok(HashMap::new()); }; let source_uids_with_multiplicity = source_uids @@ -677,6 +677,7 @@ impl IngestController { .flat_map(|(source_uid, count)| std::iter::repeat(source_uid).take(*count)); let mut init_shard_subrequests: Vec = Vec::new(); + for (subrequest_id, (source_uid, (leader_id, follower_id_opt))) in source_uids_with_multiplicity .zip(leader_follower_pairs) @@ -687,7 +688,9 @@ impl IngestController { let index_metadata = model .index_metadata(&source_uid.index_uid) .expect("index should exist"); - let doc_mapping_uid = index_metadata.index_config.doc_mapping.doc_mapping_uid; + let doc_mapping = &index_metadata.index_config.doc_mapping; + let doc_mapping_uid = doc_mapping.doc_mapping_uid; + let doc_mapping_json = serde_utils::to_json_str(doc_mapping)?; let shard = Shard { index_uid: Some(source_uid.index_uid.clone()), @@ -703,6 +706,7 @@ impl IngestController { let init_shard_subrequest = InitShardSubrequest { subrequest_id: subrequest_id as u32, shard: Some(shard), + doc_mapping_json, }; init_shard_subrequests.push(init_shard_subrequest); } @@ -729,7 +733,7 @@ impl IngestController { }) .collect(); - let OpenShardsResponse { subresponses } = progress + let open_shards_response = progress .protect_future(open_shards_on_metastore_and_model( open_shard_subrequests, &mut self.metastore, @@ -737,14 +741,14 @@ impl IngestController { )) .await?; - let mut per_source_num_open_shards: HashMap = HashMap::new(); + let mut per_source_num_opened_shards: HashMap = HashMap::new(); - for open_shard_subresponse in subresponses { + for open_shard_subresponse in open_shards_response.subresponses { let source_uid = open_shard_subresponse.open_shard().source_uid(); - *per_source_num_open_shards.entry(source_uid).or_default() += 1; + *per_source_num_opened_shards.entry(source_uid).or_default() += 1; } - Ok(per_source_num_open_shards) + Ok(per_source_num_opened_shards) } /// Attempts to decrease the number of shards. This operation is rate limited to avoid closing @@ -1139,7 +1143,7 @@ mod tests { use quickwit_actors::Universe; use quickwit_common::setup_logging_for_tests; use quickwit_common::tower::DelayLayer; - use quickwit_config::{SourceConfig, INGEST_V2_SOURCE_ID}; + use quickwit_config::{DocMapping, SourceConfig, INGEST_V2_SOURCE_ID}; use quickwit_ingest::{RateMibPerSec, ShardInfo}; use quickwit_metastore::IndexMetadata; use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; @@ -1209,7 +1213,7 @@ mod tests { let ingester = IngesterServiceClient::from_mock(mock_ingester); let ingester_pool = IngesterPool::default(); - ingester_pool.insert("test-ingester-1".into(), ingester.clone()); + ingester_pool.insert(NodeId::from("test-ingester-1"), ingester.clone()); let mut mock_ingester = MockIngesterService::new(); let index_uid_1_clone = index_uid_1.clone(); @@ -1237,10 +1241,10 @@ mod tests { Ok(response) }); let ingester = IngesterServiceClient::from_mock(mock_ingester); - ingester_pool.insert("test-ingester-2".into(), ingester.clone()); + ingester_pool.insert(NodeId::from("test-ingester-2"), ingester.clone()); let replication_factor = 2; - let mut ingest_controller = + let mut controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let mut model = ControlPlaneModel::default(); @@ -1283,7 +1287,7 @@ mod tests { closed_shards: Vec::new(), unavailable_leaders: Vec::new(), }; - let response = ingest_controller + let response = controller .get_or_create_open_shards(request, &mut model, &progress) .await .unwrap(); @@ -1320,7 +1324,7 @@ mod tests { closed_shards, unavailable_leaders, }; - let response = ingest_controller + let response = controller .get_or_create_open_shards(request, &mut model, &progress) .await .unwrap(); @@ -1418,11 +1422,10 @@ mod tests { let ingester = IngesterServiceClient::from_mock(mock_ingester); let ingester_pool = IngesterPool::default(); - ingester_pool.insert("test-ingester-1".into(), ingester.clone()); + ingester_pool.insert(NodeId::from("test-ingester-1"), ingester.clone()); let replication_factor = 1; - let mut ingest_controller = - IngestController::new(metastore, ingester_pool, replication_factor); + let mut controller = IngestController::new(metastore, ingester_pool, replication_factor); let mut model = ControlPlaneModel::default(); model.add_index(index_metadata_0.clone()); @@ -1445,7 +1448,7 @@ mod tests { unavailable_leaders: Vec::new(), }; - let metastore_error = ingest_controller + let metastore_error = controller .get_or_create_open_shards(request, &mut model, &progress) .await .unwrap_err(); @@ -1459,12 +1462,11 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 2; - let mut ingest_controller = - IngestController::new(metastore, ingester_pool, replication_factor); + let mut controller = IngestController::new(metastore, ingester_pool, replication_factor); let mut model = ControlPlaneModel::default(); let index_uid = IndexUid::for_test("test-index-0", 0); - let source_id: SourceId = "test-source".into(); + let source_id: SourceId = "test-source".to_string(); let shards = vec![Shard { shard_id: Some(ShardId::from(1)), @@ -1487,7 +1489,7 @@ mod tests { }; let progress = Progress::default(); - ingest_controller + controller .get_or_create_open_shards(request, &mut model, &progress) .await .unwrap(); @@ -1505,35 +1507,32 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 2; - let ingest_controller = + let controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let mut model = ControlPlaneModel::default(); let leader_follower_pairs_opt = - ingest_controller.allocate_shards(0, &FnvHashSet::default(), &model); + controller.allocate_shards(0, &FnvHashSet::default(), &model); assert!(leader_follower_pairs_opt.is_none()); ingester_pool.insert( - "test-ingester-1".into(), - IngesterServiceClient::from_mock(MockIngesterService::new()), + NodeId::from("test-ingester-1"), + IngesterServiceClient::mocked(), ); let leader_follower_pairs_opt = - ingest_controller.allocate_shards(0, &FnvHashSet::default(), &model); + controller.allocate_shards(0, &FnvHashSet::default(), &model); assert!(leader_follower_pairs_opt.is_none()); - ingester_pool.insert( - "test-ingester-2".into(), - IngesterServiceClient::from_mock(MockIngesterService::new()), - ); + ingester_pool.insert("test-ingester-2".into(), IngesterServiceClient::mocked()); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(0, &FnvHashSet::default(), &model) .unwrap(); assert!(leader_follower_pairs.is_empty()); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(1, &FnvHashSet::default(), &model) .unwrap(); assert_eq!(leader_follower_pairs.len(), 1); @@ -1543,7 +1542,7 @@ mod tests { Some(NodeId::from("test-ingester-2")) ); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(2, &FnvHashSet::default(), &model) .unwrap(); assert_eq!(leader_follower_pairs.len(), 2); @@ -1559,7 +1558,7 @@ mod tests { Some(NodeId::from("test-ingester-1")) ); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(3, &FnvHashSet::default(), &model) .unwrap(); assert_eq!(leader_follower_pairs.len(), 3); @@ -1582,7 +1581,7 @@ mod tests { ); let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id: SourceId = "test-source".to_string(); let open_shards = vec![Shard { index_uid: Some(index_uid.clone()), source_id: source_id.clone(), @@ -1593,7 +1592,7 @@ mod tests { }]; model.insert_shards(&index_uid, &source_id, open_shards); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(3, &FnvHashSet::default(), &model) .unwrap(); assert_eq!(leader_follower_pairs.len(), 3); @@ -1635,7 +1634,7 @@ mod tests { ]; model.insert_shards(&index_uid, &source_id, open_shards); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(1, &FnvHashSet::default(), &model) .unwrap(); assert_eq!(leader_follower_pairs.len(), 1); @@ -1645,12 +1644,9 @@ mod tests { Some(NodeId::from("test-ingester-1")) ); - ingester_pool.insert( - "test-ingester-3".into(), - IngesterServiceClient::from_mock(MockIngesterService::new()), - ); + ingester_pool.insert("test-ingester-3".into(), IngesterServiceClient::mocked()); let unavailable_leaders = FnvHashSet::from_iter([NodeId::from("test-ingester-2")]); - let leader_follower_pairs = ingest_controller + let leader_follower_pairs = controller .allocate_shards(4, &unavailable_leaders, &model) .unwrap(); assert_eq!(leader_follower_pairs.len(), 4); @@ -1685,7 +1681,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let ingest_controller = + let controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let ingester_id_0 = NodeId::from("test-ingester-0"); @@ -1768,7 +1764,7 @@ mod tests { .build_from_mock(mock_ingester_2); ingester_pool.insert(ingester_id_2, ingester_2); - let init_shards_response = ingest_controller + let init_shards_response = controller .init_shards(Vec::new(), &Progress::default()) .await; assert_eq!(init_shards_response.successes.len(), 0); @@ -1791,6 +1787,7 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }), + doc_mapping_json: "{}".to_string(), }, InitShardSubrequest { subrequest_id: 1, @@ -1802,6 +1799,7 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }), + doc_mapping_json: "{}".to_string(), }, InitShardSubrequest { subrequest_id: 2, @@ -1813,6 +1811,7 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }), + doc_mapping_json: "{}".to_string(), }, InitShardSubrequest { subrequest_id: 3, @@ -1824,6 +1823,7 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }), + doc_mapping_json: "{}".to_string(), }, InitShardSubrequest { subrequest_id: 4, @@ -1835,9 +1835,10 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }), + doc_mapping_json: "{}".to_string(), }, ]; - let init_shards_response = ingest_controller + let init_shards_response = controller .init_shards(init_shard_subrequests, &Progress::default()) .await; assert_eq!(init_shards_response.successes.len(), 1); @@ -1855,6 +1856,121 @@ mod tests { assert_eq!(failures[3].subrequest_id, 4); } + #[tokio::test] + async fn test_ingest_controller_try_open_shards() { + let doc_mapping_uid = DocMappingUid::random(); + let expected_doc_mapping = doc_mapping_uid; + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_open_shards() + .once() + .returning(move |request| { + assert_eq!(request.subrequests.len(), 1); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); + + assert_eq!(subrequest.index_uid(), &("test-index", 0)); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.leader_id, "test-ingester-1"); + assert_eq!(subrequest.doc_mapping_uid(), expected_doc_mapping); + + let subresponses = vec![metastore::OpenShardSubresponse { + subrequest_id: 0, + open_shard: Some(Shard { + index_uid: Some(IndexUid::for_test("test-index", 0)), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(0)), + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(expected_doc_mapping), + ..Default::default() + }), + }]; + let response = metastore::OpenShardsResponse { subresponses }; + Ok(response) + }); + let metastore = MetastoreServiceClient::from_mock(mock_metastore); + let ingester_pool = IngesterPool::default(); + let replication_factor = 1; + + let mut controller = + IngestController::new(metastore, ingester_pool.clone(), replication_factor); + + let index_uid = IndexUid::for_test("test-index", 0); + let source_id = "test-source".to_string(); + + let source_uid = SourceUid { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + }; + let mut index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); + + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}", + "field_mappings": [{{ + "name": "message", + "type": "text" + }}] + }}"# + ); + let doc_mapping: DocMapping = serde_json::from_str(&doc_mapping_json).unwrap(); + let expected_doc_mapping = doc_mapping.clone(); + index_metadata.index_config.doc_mapping = doc_mapping; + + let mut model = ControlPlaneModel::default(); + model.add_index(index_metadata); + + let mut mock_ingester = MockIngesterService::new(); + mock_ingester + .expect_init_shards() + .once() + .returning(move |request| { + assert_eq!(request.subrequests.len(), 1); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); + + let doc_mapping: DocMapping = + serde_json::from_str(&subrequest.doc_mapping_json).unwrap(); + assert_eq!(doc_mapping, expected_doc_mapping); + + let shard = request.subrequests[0].shard(); + assert_eq!(shard.index_uid(), &("test-index", 0)); + assert_eq!(shard.source_id, "test-source"); + assert_eq!(shard.leader_id, "test-ingester-1"); + assert_eq!(shard.doc_mapping_uid(), doc_mapping_uid); + + let successes = vec![InitShardSuccess { + subrequest_id: 0, + shard: Some(shard.clone()), + }]; + let response = InitShardsResponse { + successes, + failures: Vec::new(), + }; + Ok(response) + }); + + ingester_pool.insert( + NodeId::from("test-ingester-1"), + IngesterServiceClient::from_mock(mock_ingester), + ); + let source_uids: HashMap = HashMap::from_iter([(source_uid.clone(), 1)]); + let unavailable_leaders = FnvHashSet::default(); + let progress = Progress::default(); + + let per_source_num_opened_shards = controller + .try_open_shards(source_uids, &mut model, &unavailable_leaders, &progress) + .await + .unwrap(); + + assert_eq!(per_source_num_opened_shards.len(), 1); + assert_eq!(*per_source_num_opened_shards.get(&source_uid).unwrap(), 1); + } + #[tokio::test] async fn test_ingest_controller_handle_local_shards_update() { let mut mock_metastore = MockMetastoreService::new(); @@ -1907,12 +2023,12 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut ingest_controller = + let mut controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let index_uid = IndexUid::for_test("test-index", 0); let index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); - let source_id: SourceId = "test-source".into(); + let source_id: SourceId = "test-source".to_string(); let source_uid = SourceUid { index_uid: index_uid.clone(), @@ -1948,7 +2064,7 @@ mod tests { shard_infos, }; - ingest_controller + controller .handle_local_shards_update(local_shards_update, &mut model, &progress) .await .unwrap(); @@ -2020,7 +2136,7 @@ mod tests { source_uid: source_uid.clone(), shard_infos, }; - ingest_controller + controller .handle_local_shards_update(local_shards_update, &mut model, &progress) .await .unwrap(); @@ -2045,7 +2161,7 @@ mod tests { }; // The first request fails due to an error on the metastore. - let MetastoreError::InvalidArgument { .. } = ingest_controller + let MetastoreError::InvalidArgument { .. } = controller .handle_local_shards_update(local_shards_update.clone(), &mut model, &progress) .await .unwrap_err() @@ -2054,7 +2170,7 @@ mod tests { }; // The second request works! - ingest_controller + controller .handle_local_shards_update(local_shards_update, &mut model, &progress) .await .unwrap(); @@ -2107,7 +2223,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut ingest_controller = + let mut controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let index_uid = IndexUid::for_test("test-index", 0); @@ -2132,7 +2248,7 @@ mod tests { let progress = Progress::default(); // Test could not find leader. - ingest_controller + controller .try_scale_up_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); @@ -2184,21 +2300,21 @@ mod tests { ingester_pool.insert("test-ingester".into(), ingester); // Test failed to open shards. - ingest_controller + controller .try_scale_up_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); assert_eq!(model.all_shards().count(), 0); // Test failed to init shards. - ingest_controller + controller .try_scale_up_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap_err(); assert_eq!(model.all_shards().count(), 0); // Test successfully opened shard. - ingest_controller + controller .try_scale_up_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); @@ -2214,11 +2330,11 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let ingest_controller = + let controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id: SourceId = "test-source".to_string(); let source_uid = SourceUid { index_uid: index_uid.clone(), @@ -2232,7 +2348,7 @@ mod tests { let progress = Progress::default(); // Test could not find a scale down candidate. - ingest_controller + controller .try_scale_down_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); @@ -2248,7 +2364,7 @@ mod tests { model.insert_shards(&index_uid, &source_id, shards); // Test ingester is unavailable. - ingest_controller + controller .try_scale_down_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); @@ -2288,14 +2404,14 @@ mod tests { ingester_pool.insert("test-ingester".into(), ingester); // Test failed to close shard. - ingest_controller + controller .try_scale_down_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); assert!(model.all_shards().all(|shard| shard.is_open())); // Test successfully closed shard. - ingest_controller + controller .try_scale_down_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); @@ -2312,7 +2428,7 @@ mod tests { model.insert_shards(&index_uid, &source_id, shards); // Test rate limited. - ingest_controller + controller .try_scale_down_shards(source_uid.clone(), shard_stats, &mut model, &progress) .await .unwrap(); @@ -2322,7 +2438,7 @@ mod tests { #[test] fn test_find_scale_down_candidate() { let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id: SourceId = "test-source".to_string(); let source_uid = SourceUid { index_uid: index_uid.clone(), @@ -2429,11 +2545,11 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 2; - let ingest_controller = + let controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id: SourceId = "test-source".to_string(); let mut model = ControlPlaneModel::default(); let shards = vec![ Shard { @@ -2497,7 +2613,7 @@ mod tests { IngesterServiceClient::from_mock(mock_ingester_3), ); let node_id = "node-1".into(); - let wait_handle = ingest_controller.sync_with_ingester(&node_id, &model); + let wait_handle = controller.sync_with_ingester(&node_id, &model); wait_handle.wait().await; assert_eq!(count_calls.load(Ordering::Acquire), 1); } @@ -2508,7 +2624,7 @@ mod tests { let ingester_pool = IngesterPool::default(); let replication_factor = 2; - let ingest_controller = IngestController::new(metastore, ingester_pool, replication_factor); + let controller = IngestController::new(metastore, ingester_pool, replication_factor); let mut model = ControlPlaneModel::default(); @@ -2541,7 +2657,7 @@ mod tests { ], }; let advise_reset_shards_response = - ingest_controller.advise_reset_shards(advise_reset_shards_request, &model); + controller.advise_reset_shards(advise_reset_shards_request, &model); assert_eq!(advise_reset_shards_response.shards_to_delete.len(), 2); @@ -2578,10 +2694,10 @@ mod tests { let metastore = MetastoreServiceClient::mocked(); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let ingest_controller = + let controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); - let closed_shards = ingest_controller.close_shards(Vec::new()).await; + let closed_shards = controller.close_shards(Vec::new()).await; assert_eq!(closed_shards.len(), 0); let ingester_id_0 = NodeId::from("test-ingester-0"); @@ -2689,7 +2805,7 @@ mod tests { }, ), ]; - let closed_shards = ingest_controller.close_shards(shards_to_close).await; + let closed_shards = controller.close_shards(shards_to_close).await; assert_eq!(closed_shards.len(), 1); let closed_shard = &closed_shards[0]; @@ -2730,7 +2846,7 @@ mod tests { let metastore = MetastoreServiceClient::from_mock(mock_metastore); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut ingest_controller = + let mut controller = IngestController::new(metastore, ingester_pool.clone(), replication_factor); let mut model = ControlPlaneModel::default(); @@ -2739,7 +2855,7 @@ mod tests { let (control_plane_mailbox, control_plane_inbox) = universe.create_test_mailbox(); let progress = Progress::default(); - let close_shards_task_opt = ingest_controller + let close_shards_task_opt = controller .rebalance_shards(&mut model, &control_plane_mailbox, &progress) .await .unwrap(); @@ -2863,7 +2979,7 @@ mod tests { let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); ingester_pool.insert(ingester_id_1.clone(), ingester_1); - let close_shards_task = ingest_controller + let close_shards_task = controller .rebalance_shards(&mut model, &control_plane_mailbox, &progress) .await .unwrap() diff --git a/quickwit/quickwit-indexing/src/actors/doc_processor.rs b/quickwit/quickwit-indexing/src/actors/doc_processor.rs index 7eacb91869e..31d6f25efd6 100644 --- a/quickwit/quickwit-indexing/src/actors/doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/doc_processor.rs @@ -324,7 +324,7 @@ pub struct DocProcessorCounters { /// - number of docs that could not be parsed. /// - number of docs that were not valid json. /// - number of docs that could not be transformed. - /// - number of docs for which the doc mapper returnd an error. + /// - number of docs for which the doc mapper returned an error. /// - number of valid docs. pub valid: DocProcessorCounter, pub doc_mapper_errors: DocProcessorCounter, diff --git a/quickwit/quickwit-ingest/Cargo.toml b/quickwit/quickwit-ingest/Cargo.toml index 0e0860028d0..17502a1f0a7 100644 --- a/quickwit/quickwit-ingest/Cargo.toml +++ b/quickwit/quickwit-ingest/Cargo.toml @@ -38,6 +38,7 @@ quickwit-actors = { workspace = true } quickwit-cluster = { workspace = true } quickwit-common = { workspace = true } quickwit-config = { workspace = true } +quickwit-doc-mapper = { workspace = true, features = ["testsuite"] } quickwit-proto = { workspace = true } [dev-dependencies] diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs index 9fa8d8d930a..36b91e8fced 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs @@ -484,6 +484,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, Instant::now(), ); state_guard.shards.insert(queue_id_00.clone(), shard_00); @@ -493,6 +494,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, Instant::now(), ); shard_01.is_advertisable = true; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs b/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs new file mode 100644 index 00000000000..938b3b330d8 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/doc_mapper.rs @@ -0,0 +1,243 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::collections::hash_map::Entry; +use std::collections::HashMap; +use std::sync::{Arc, Weak}; + +use quickwit_common::thread_pool::run_cpu_intensive; +use quickwit_config::{build_doc_mapper, DocMapping, SearchSettings}; +use quickwit_doc_mapper::DocMapper; +use quickwit_proto::ingest::{ + DocBatchV2, IngestV2Error, IngestV2Result, ParseFailure, ParseFailureReason, +}; +use quickwit_proto::types::DocMappingUid; +use serde_json::Value as JsonValue; +use tracing::info; + +/// Attempts to get the doc mapper identified by the given doc mapping UID `doc_mapping_uid` from +/// the `doc_mappers` cache. If it is not found, it is built from the specified JSON doc mapping +/// `doc_mapping_json` and inserted into the cache before being returned. +pub(super) fn get_or_try_build_doc_mapper( + doc_mappers: &mut HashMap>, + doc_mapping_uid: DocMappingUid, + doc_mapping_json: &str, +) -> IngestV2Result> { + if let Entry::Occupied(occupied) = doc_mappers.entry(doc_mapping_uid) { + if let Some(doc_mapper) = occupied.get().upgrade() { + return Ok(doc_mapper); + } + occupied.remove(); + } + let doc_mapper = try_build_doc_mapper(doc_mapping_json)?; + + if doc_mapper.doc_mapping_uid() != doc_mapping_uid { + let message = format!( + "doc mapping UID mismatch: expected `{doc_mapping_uid}`, got `{}`", + doc_mapper.doc_mapping_uid() + ); + return Err(IngestV2Error::Internal(message)); + } + doc_mappers.insert(doc_mapping_uid, Arc::downgrade(&doc_mapper)); + info!("inserted doc mapper `{doc_mapping_uid}` into cache`"); + + Ok(doc_mapper) +} + +/// Attempts to build a doc mapper from the specified JSON doc mapping `doc_mapping_json`. +pub(super) fn try_build_doc_mapper(doc_mapping_json: &str) -> IngestV2Result> { + let doc_mapping: DocMapping = serde_json::from_str(doc_mapping_json).map_err(|error| { + IngestV2Error::Internal(format!("failed to parse doc mapping: {error}")) + })?; + let search_settings = SearchSettings::default(); + let doc_mapper = build_doc_mapper(&doc_mapping, &search_settings) + .map_err(|error| IngestV2Error::Internal(format!("failed to build doc mapper: {error}")))?; + Ok(doc_mapper) +} + +/// Parses the JSON documents contained in the batch and applies the doc mapper. Returns the +/// original batch and a list of parse failures. +pub(super) async fn validate_doc_batch( + doc_batch: DocBatchV2, + doc_mapper: Arc, +) -> IngestV2Result<(DocBatchV2, Vec)> { + let validate_doc_batch_fn = move || { + let mut parse_failures: Vec = Vec::new(); + + for (doc_uid, doc) in doc_batch.docs() { + let Ok(json_doc) = serde_json::from_slice::(&doc) else { + let parse_failure = ParseFailure { + doc_uid: Some(doc_uid), + reason: ParseFailureReason::InvalidJson as i32, + message: "failed to parse JSON document".to_string(), + }; + parse_failures.push(parse_failure); + continue; + }; + let JsonValue::Object(json_obj) = json_doc else { + let parse_failure = ParseFailure { + doc_uid: Some(doc_uid), + reason: ParseFailureReason::InvalidJson as i32, + message: "JSON document is not an object".to_string(), + }; + parse_failures.push(parse_failure); + continue; + }; + if let Err(error) = doc_mapper.doc_from_json_obj(json_obj, doc.len() as u64) { + let parse_failure = ParseFailure { + doc_uid: Some(doc_uid), + reason: ParseFailureReason::InvalidSchema as i32, + message: error.to_string(), + }; + parse_failures.push(parse_failure); + } + } + (doc_batch, parse_failures) + }; + run_cpu_intensive(validate_doc_batch_fn) + .await + .map_err(|error| { + let message = format!("failed to validate documents: {error}"); + IngestV2Error::Internal(message) + }) +} + +#[cfg(test)] +mod tests { + use quickwit_proto::types::DocUid; + + use super::*; + + #[test] + fn test_get_or_try_build_doc_mapper() { + let mut doc_mappers: HashMap> = HashMap::new(); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = r#"{ + "field_mappings": [{ + "name": "message", + "type": "text" + }] + }"#; + let error = + get_or_try_build_doc_mapper(&mut doc_mappers, doc_mapping_uid, doc_mapping_json) + .unwrap_err(); + assert!( + matches!(error, IngestV2Error::Internal(message) if message.contains("doc mapping UID mismatch")) + ); + + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}", + "field_mappings": [{{ + "name": "message", + "type": "text" + }}] + }}"# + ); + let doc_mapper = + get_or_try_build_doc_mapper(&mut doc_mappers, doc_mapping_uid, &doc_mapping_json) + .unwrap(); + assert_eq!(doc_mappers.len(), 1); + assert_eq!(doc_mapper.doc_mapping_uid(), doc_mapping_uid); + assert_eq!(Arc::strong_count(&doc_mapper), 1); + + drop(doc_mapper); + assert!(doc_mappers + .get(&doc_mapping_uid) + .unwrap() + .upgrade() + .is_none()); + + let error = get_or_try_build_doc_mapper(&mut doc_mappers, doc_mapping_uid, "").unwrap_err(); + assert!( + matches!(error, IngestV2Error::Internal(message) if message.contains("parse doc mapping")) + ); + assert_eq!(doc_mappers.len(), 0); + } + + #[test] + fn test_try_build_doc_mapper() { + let error = try_build_doc_mapper("").unwrap_err(); + assert!( + matches!(error, IngestV2Error::Internal(message) if message.contains("parse doc mapping")) + ); + + let error = try_build_doc_mapper(r#"{"timestamp_field": ".timestamp"}"#).unwrap_err(); + assert!( + matches!(error, IngestV2Error::Internal(message) if message.contains("build doc mapper")) + ); + + let doc_mapping_json = r#"{ + "mode": "strict", + "field_mappings": [{ + "name": "message", + "type": "text" + }]}"#; + let doc_mapper = try_build_doc_mapper(doc_mapping_json).unwrap(); + let schema = doc_mapper.schema(); + assert_eq!(schema.num_fields(), 2); + + let contains_message_field = schema + .fields() + .map(|(_field, entry)| entry.name()) + .any(|field_name| field_name == "message"); + assert!(contains_message_field); + } + + #[tokio::test] + async fn test_validate_doc_batch() { + let doc_mapping_json = r#"{ + "mode": "strict", + "field_mappings": [ + { + "name": "doc", + "type": "text" + } + ] + }"#; + let doc_mapper = try_build_doc_mapper(doc_mapping_json).unwrap(); + let doc_batch = DocBatchV2::default(); + + let (_, parse_failures) = validate_doc_batch(doc_batch, doc_mapper.clone()) + .await + .unwrap(); + assert_eq!(parse_failures.len(), 0); + + let doc_batch = + DocBatchV2::for_test(["", "[]", r#"{"foo": "bar"}"#, r#"{"doc": "test-doc-000"}"#]); + let (_, parse_failures) = validate_doc_batch(doc_batch, doc_mapper).await.unwrap(); + assert_eq!(parse_failures.len(), 3); + + let parse_failure_0 = &parse_failures[0]; + assert_eq!(parse_failure_0.doc_uid(), DocUid::for_test(0)); + assert_eq!(parse_failure_0.reason(), ParseFailureReason::InvalidJson); + assert!(parse_failure_0.message.contains("parse JSON document")); + + let parse_failure_1 = &parse_failures[1]; + assert_eq!(parse_failure_1.doc_uid(), DocUid::for_test(1)); + assert_eq!(parse_failure_1.reason(), ParseFailureReason::InvalidJson); + assert!(parse_failure_1.message.contains("not an object")); + + let parse_failure_2 = &parse_failures[2]; + assert_eq!(parse_failure_2.doc_uid(), DocUid::for_test(2)); + assert_eq!(parse_failure_2.reason(), ParseFailureReason::InvalidSchema); + assert!(parse_failure_2.message.contains("not declared")); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs index 26e7078ba97..0263e23d194 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs @@ -105,6 +105,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, now - idle_shard_timeout, ); let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); @@ -114,6 +115,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, now - idle_shard_timeout / 2, ); let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index 7750e6f5742..94f5a62464d 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -53,10 +53,11 @@ use quickwit_proto::ingest::ingester::{ SynReplicationMessage, TruncateShardsRequest, TruncateShardsResponse, }; use quickwit_proto::ingest::{ - CommitTypeV2, IngestV2Error, IngestV2Result, Shard, ShardIds, ShardState, + CommitTypeV2, DocBatchV2, IngestV2Error, IngestV2Result, ParseFailure, Shard, ShardIds, + ShardState, }; use quickwit_proto::types::{ - queue_id, split_queue_id, IndexUid, NodeId, Position, QueueId, ShardId, SourceId, + queue_id, split_queue_id, IndexUid, NodeId, Position, QueueId, ShardId, SourceId, SubrequestId, }; use serde_json::{json, Value as JsonValue}; use tokio::sync::Semaphore; @@ -64,6 +65,7 @@ use tokio::time::{sleep, timeout}; use tracing::{debug, error, info, warn}; use super::broadcast::BroadcastLocalShardsTask; +use super::doc_mapper::validate_doc_batch; use super::fetch::FetchStreamTask; use super::idle::CloseIdleShardsTask; use super::metrics::INGEST_V2_METRICS; @@ -78,6 +80,7 @@ use super::replication::{ }; use super::state::{IngesterState, InnerIngesterState, WeakIngesterState}; use super::IngesterPool; +use crate::ingest_v2::doc_mapper::get_or_try_build_doc_mapper; use crate::ingest_v2::metrics::report_wal_usage; use crate::ingest_v2::models::IngesterShardType; use crate::metrics::INGEST_METRICS; @@ -188,6 +191,7 @@ impl Ingester { state: &mut InnerIngesterState, mrecordlog: &mut MultiRecordLogAsync, shard: Shard, + doc_mapping_json: &str, now: Instant, ) -> IngestV2Result<()> { let queue_id = shard.queue_id(); @@ -200,6 +204,11 @@ impl Ingester { let Entry::Vacant(entry) = state.shards.entry(queue_id.clone()) else { return Ok(()); }; + let doc_mapper = get_or_try_build_doc_mapper( + &mut state.doc_mappers, + shard.doc_mapping_uid(), + doc_mapping_json, + )?; match mrecordlog.create_queue(&queue_id).await { Ok(_) => {} Err(CreateQueueError::AlreadyExists) => { @@ -233,16 +242,16 @@ impl Ingester { if let Err(error) = replication_client.init_replica(shard).await { // TODO: Remove dangling queue from the WAL. - error!("failed to initialize replica shard: {error}",); - return Err(IngestV2Error::Internal(format!( - "failed to initialize replica shard: {error}" - ))); + error!("failed to initialize replica shard: {error}"); + let message = format!("failed to initialize replica shard: {error}"); + return Err(IngestV2Error::Internal(message)); } IngesterShard::new_primary( follower_id, ShardState::Open, Position::Beginning, Position::Beginning, + doc_mapper, now, ) } else { @@ -250,6 +259,7 @@ impl Ingester { ShardState::Open, Position::Beginning, Position::Beginning, + Some(doc_mapper), now, ) }; @@ -433,10 +443,10 @@ impl Ingester { } let mut persist_successes = Vec::with_capacity(persist_request.subrequests.len()); let mut persist_failures = Vec::new(); - let mut replicate_subrequests: HashMap> = + let mut per_follower_replicate_subrequests: HashMap> = HashMap::new(); - let mut local_persist_subrequests: Vec = - Vec::with_capacity(persist_request.subrequests.len()); + let mut pending_persist_subrequests: HashMap = + HashMap::with_capacity(persist_request.subrequests.len()); // Keep track of the shards that need to be closed following an IO error. let mut shards_to_close: HashSet = HashSet::new(); @@ -474,7 +484,7 @@ impl Ingester { } // first verify if we would locally accept each subrequest { - let mut total_requested_capacity = bytesize::ByteSize::b(0); + let mut total_requested_capacity = ByteSize::b(0); for subrequest in persist_request.subrequests { let queue_id = subrequest.queue_id(); @@ -506,29 +516,41 @@ impl Ingester { persist_failures.push(persist_failure); continue; } - + let doc_mapper = shard.doc_mapper_opt.clone().expect("shard should be open"); let follower_id_opt = shard.follower_id_opt().cloned(); let from_position_exclusive = shard.replication_position_inclusive.clone(); - let index_uid = subrequest.index_uid().clone(); - let doc_batch = match subrequest.doc_batch { - Some(doc_batch) if !doc_batch.is_empty() => doc_batch, + let (doc_batch, parse_failures) = match subrequest.doc_batch { + Some(doc_batch) if !doc_batch.is_empty() => { + // TODO: `validate_doc_batch` could remove from the batch the documents for + // which a parse failure occurred so we don't + // persist and replicate them unnecessarily. + // However, the doc processor metrics would be off. + validate_doc_batch(doc_batch, doc_mapper).await? + } _ => { warn!("received empty persist request"); - - let persist_success = PersistSuccess { - subrequest_id: subrequest.subrequest_id, - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - replication_position_inclusive: Some( - shard.replication_position_inclusive.clone(), - ), - }; - persist_successes.push(persist_success); - continue; + (DocBatchV2::default(), Vec::new()) } }; + let num_persisted_docs = (doc_batch.num_docs() - parse_failures.len()) as u32; + + if num_persisted_docs == 0 { + let replication_position_inclusive = + Some(shard.replication_position_inclusive.clone()); + + let persist_success = PersistSuccess { + subrequest_id: subrequest.subrequest_id, + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shard_id: subrequest.shard_id, + replication_position_inclusive, + num_persisted_docs, + parse_failures, + }; + persist_successes.push(persist_success); + continue; + } let requested_capacity = estimate_size(&doc_batch); if let Err(error) = check_enough_capacity( @@ -575,58 +597,62 @@ impl Ingester { rate_meter.update(batch_num_bytes); total_requested_capacity += requested_capacity; + let mut successfully_replicated = true; + if let Some(follower_id) = follower_id_opt { + successfully_replicated = false; + let replicate_subrequest = ReplicateSubrequest { subrequest_id: subrequest.subrequest_id, - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, + index_uid: subrequest.index_uid.clone(), + source_id: subrequest.source_id.clone(), + shard_id: subrequest.shard_id.clone(), from_position_exclusive: Some(from_position_exclusive), - doc_batch: Some(doc_batch), + doc_batch: Some(doc_batch.clone()), }; - replicate_subrequests + per_follower_replicate_subrequests .entry(follower_id) .or_default() - .push((replicate_subrequest, queue_id)); - } else { - local_persist_subrequests.push(LocalPersistSubrequest { - queue_id, - subrequest_id: subrequest.subrequest_id, - index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - doc_batch, - expected_position_inclusive: None, - }) + .push(replicate_subrequest); } + let pending_persist_subrequest = PendingPersistSubrequest { + queue_id, + subrequest_id: subrequest.subrequest_id, + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shard_id: subrequest.shard_id, + doc_batch, + num_persisted_docs, + parse_failures, + expected_position_inclusive: None, + successfully_replicated, + }; + pending_persist_subrequests.insert( + pending_persist_subrequest.subrequest_id, + pending_persist_subrequest, + ); } } // replicate to the follower { let mut replicate_futures = FuturesUnordered::new(); - let mut doc_batch_map = HashMap::new(); - for (follower_id, subrequests_with_queue_id) in replicate_subrequests { + for (follower_id, replicate_subrequests) in per_follower_replicate_subrequests { let replication_client = state_guard .replication_streams .get(&follower_id) .expect("replication stream should be initialized") .replication_client(); let leader_id = self.self_node_id.clone(); - let mut subrequests = Vec::with_capacity(subrequests_with_queue_id.len()); - for (subrequest, queue_id) in subrequests_with_queue_id { - let doc_batch = subrequest - .doc_batch - .clone() - .expect("we already verified doc is present and not empty"); - doc_batch_map.insert(subrequest.subrequest_id, (doc_batch, queue_id)); - subrequests.push(subrequest); - } - let replicate_future = - replication_client.replicate(leader_id, follower_id, subrequests, commit_type); + + let replicate_future = replication_client.replicate( + leader_id, + follower_id, + replicate_subrequests, + commit_type, + ); replicate_futures.push(replicate_future); } - while let Some(replication_result) = replicate_futures.next().await { let replicate_response = match replication_result { Ok(replicate_response) => replicate_response, @@ -639,20 +665,13 @@ impl Ingester { } }; for replicate_success in replicate_response.successes { - let (doc_batch, queue_id) = doc_batch_map - .remove(&replicate_success.subrequest_id) - .expect("expected known subrequest id"); - let local_persist_subrequest = LocalPersistSubrequest { - queue_id, - subrequest_id: replicate_success.subrequest_id, - index_uid: replicate_success.index_uid().clone(), - source_id: replicate_success.source_id, - shard_id: replicate_success.shard_id, - doc_batch, - expected_position_inclusive: replicate_success - .replication_position_inclusive, - }; - local_persist_subrequests.push(local_persist_subrequest); + let pending_persist_subrequest = pending_persist_subrequests + .get_mut(&replicate_success.subrequest_id) + .expect("persist subrequest should exist"); + + pending_persist_subrequest.successfully_replicated = true; + pending_persist_subrequest.expected_position_inclusive = + replicate_success.replication_position_inclusive; } for replicate_failure in replicate_response.failures { // TODO: If the replica shard is closed, close the primary shard if it is not @@ -678,11 +697,13 @@ impl Ingester { } } } - // finally write locally { let now = Instant::now(); - for subrequest in local_persist_subrequests { + for subrequest in pending_persist_subrequests.into_values() { + if !subrequest.successfully_replicated { + continue; + } let queue_id = subrequest.queue_id; let batch_num_bytes = subrequest.doc_batch.num_bytes() as u64; @@ -718,7 +739,7 @@ impl Ingester { }; let persist_failure = PersistFailure { subrequest_id: subrequest.subrequest_id, - index_uid: Some(subrequest.index_uid), + index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, reason: reason as i32, @@ -747,10 +768,12 @@ impl Ingester { let persist_success = PersistSuccess { subrequest_id: subrequest.subrequest_id, - index_uid: Some(subrequest.index_uid), + index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, replication_position_inclusive: Some(current_position_inclusive), + num_persisted_docs: subrequest.num_persisted_docs, + parse_failures: subrequest.parse_failures, }; persist_successes.push(persist_success); } @@ -783,6 +806,11 @@ impl Ingester { } report_wal_usage(wal_usage); + #[cfg(test)] + { + persist_successes.sort_by_key(|success| success.subrequest_id); + persist_failures.sort_by_key(|failure| failure.subrequest_id); + } let leader_id = self.self_node_id.to_string(); let persist_response = PersistResponse { leader_id, @@ -911,6 +939,7 @@ impl Ingester { &mut state_guard.inner, &mut state_guard.mrecordlog, subrequest.shard().clone(), + &subrequest.doc_mapping_json, now, ) .await; @@ -1035,7 +1064,7 @@ impl Ingester { "truncation_position_inclusive": shard.truncation_position_inclusive, }); match &shard.shard_type { - IngesterShardType::Primary { follower_id } => { + IngesterShardType::Primary { follower_id, .. } => { shard_json["type"] = json!("primary"); shard_json["leader_id"] = json!(self.self_node_id.to_string()); shard_json["follower_id"] = json!(follower_id.to_string()); @@ -1232,14 +1261,17 @@ pub async fn wait_for_ingester_decommission(ingester: Ingester) -> anyhow::Resul Ok(()) } -struct LocalPersistSubrequest { +struct PendingPersistSubrequest { queue_id: QueueId, subrequest_id: u32, - index_uid: IndexUid, + index_uid: Option, source_id: SourceId, - shard_id: Option, - doc_batch: quickwit_proto::ingest::DocBatchV2, + shard_id: Option, + doc_batch: DocBatchV2, + num_persisted_docs: u32, + parse_failures: Vec, expected_position_inclusive: Option, + successfully_replicated: bool, } #[cfg(test)] @@ -1261,15 +1293,16 @@ mod tests { PersistSubrequest, TruncateShardsSubrequest, }; use quickwit_proto::ingest::{ - DocBatchV2, ShardIdPosition, ShardIdPositions, ShardIds, ShardPKey, + DocBatchV2, ParseFailureReason, ShardIdPosition, ShardIdPositions, ShardIds, ShardPKey, }; - use quickwit_proto::types::{queue_id, DocMappingUid, ShardId, SourceUid}; + use quickwit_proto::types::{queue_id, DocMappingUid, DocUid, ShardId, SourceUid}; use tokio::task::yield_now; use tokio::time::timeout; use tonic::transport::{Endpoint, Server}; use super::*; use crate::ingest_v2::broadcast::ShardInfos; + use crate::ingest_v2::doc_mapper::try_build_doc_mapper; use crate::ingest_v2::fetch::tests::{into_fetch_eof, into_fetch_payload}; use crate::ingest_v2::DEFAULT_IDLE_SHARD_TIMEOUT; use crate::MRecord; @@ -1487,7 +1520,7 @@ mod tests { state_guard .mrecordlog - .assert_records_eq(&queue_id_02, .., &[(1, "\0\0test-doc-bar")]); + .assert_records_eq(&queue_id_02, .., &[(1, [0, 0], "test-doc-bar")]); state_guard.rate_trackers.contains_key(&queue_id_02); @@ -1506,6 +1539,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, Instant::now(), ); state_guard.shards.insert(queue_id_00.clone(), shard_00); @@ -1515,6 +1549,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, Instant::now(), ); shard_01.is_advertisable = true; @@ -1573,11 +1608,65 @@ mod tests { assert!(value_opt.is_none()); } + #[tokio::test] + async fn test_ingester_init_primary_shard() { + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; + + let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}", + "field_mappings": [{{ + "name": "message", + "type": "text" + }}] + }}"# + ); + let primary_shard = Shard { + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(1)), + shard_state: ShardState::Open as i32, + leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), + ..Default::default() + }; + let mut state_guard = ingester.state.lock_fully().await.unwrap(); + + ingester + .init_primary_shard( + &mut state_guard.inner, + &mut state_guard.mrecordlog, + primary_shard, + &doc_mapping_json, + Instant::now(), + ) + .await + .unwrap(); + + let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let shard = state_guard.shards.get(&queue_id).unwrap(); + shard.assert_is_solo(); + shard.assert_is_open(); + shard.assert_replication_position(Position::Beginning); + shard.assert_truncation_position(Position::Beginning); + assert!(shard.doc_mapper_opt.is_some()); + } + #[tokio::test] async fn test_ingester_init_shards() { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), @@ -1585,7 +1674,7 @@ mod tests { shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), follower_id: None, - doc_mapping_uid: Some(DocMappingUid::default()), + doc_mapping_uid: Some(doc_mapping_uid), publish_position_inclusive: None, publish_token: None, }; @@ -1593,6 +1682,7 @@ mod tests { subrequests: vec![InitShardSubrequest { subrequest_id: 0, shard: Some(shard.clone()), + doc_mapping_json, }], }; let response = ingester.init_shards(init_shards_request).await.unwrap(); @@ -1622,6 +1712,13 @@ mod tests { let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let init_shards_request = InitShardsRequest { subrequests: vec![ InitShardSubrequest { @@ -1632,8 +1729,10 @@ mod tests { shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json: doc_mapping_json.clone(), }, InitShardSubrequest { subrequest_id: 1, @@ -1643,8 +1742,10 @@ mod tests { shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json, }, ], }; @@ -1659,14 +1760,17 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-110", "test-doc-111"])), + doc_batch: Some(DocBatchV2::for_test([ + r#"{"doc": "test-doc-110"}"#, + r#"{"doc": "test-doc-111"}"#, + ])), }, ], }; @@ -1707,7 +1811,7 @@ mod tests { state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010"), (1, "\0\x01")], + &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#), (1, [0, 1], "")], ); let queue_id_11 = queue_id(&index_uid2, "test-source", &ShardId::from(1)); @@ -1720,9 +1824,9 @@ mod tests { &queue_id_11, .., &[ - (0, "\0\0test-doc-110"), - (1, "\0\0test-doc-111"), - (2, "\0\x01"), + (0, [0, 0], r#"{"doc": "test-doc-110"}"#), + (1, [0, 0], r#"{"doc": "test-doc-111"}"#), + (2, [0, 1], ""), ], ); } @@ -1732,6 +1836,32 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); + let init_shards_request = InitShardsRequest { + subrequests: vec![InitShardSubrequest { + subrequest_id: 0, + shard: Some(Shard { + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(0)), + shard_state: ShardState::Open as i32, + leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), + ..Default::default() + }), + doc_mapping_json, + }], + }; + let response = ingester.init_shards(init_shards_request).await.unwrap(); + assert_eq!(response.successes.len(), 1); + assert_eq!(response.failures.len(), 0); + let persist_request = PersistRequest { leader_id: ingester_ctx.node_id.to_string(), commit_type: CommitTypeV2::Force as i32, @@ -1749,40 +1879,98 @@ mod tests { subrequest_id: 0, index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), + shard_id: Some(ShardId::from(0)), doc_batch: None, }], }; + let persist_response = ingester.persist(persist_request).await.unwrap(); + assert_eq!(persist_response.leader_id, "test-ingester"); + assert_eq!(persist_response.successes.len(), 1); + assert_eq!(persist_response.failures.len(), 0); + + let persist_success = &persist_response.successes[0]; + assert_eq!(persist_success.subrequest_id, 0); + assert_eq!(persist_success.index_uid(), &index_uid); + assert_eq!(persist_success.source_id, "test-source"); + assert_eq!(persist_success.shard_id(), ShardId::from(0)); + assert_eq!( + persist_success.replication_position_inclusive, + Some(Position::Beginning) + ); + } + #[tokio::test] + async fn test_ingester_persist_validates_docs() { + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; + + let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}", + "mode": "strict", + "field_mappings": [{{"name": "doc", "type": "text"}}] + }}"# + ); let init_shards_request = InitShardsRequest { subrequests: vec![InitShardSubrequest { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), + shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json, }], }; - ingester.init_shards(init_shards_request).await.unwrap(); + let response = ingester.init_shards(init_shards_request).await.unwrap(); + assert_eq!(response.successes.len(), 1); + assert_eq!(response.failures.len(), 0); + let persist_request = PersistRequest { + leader_id: ingester_ctx.node_id.to_string(), + commit_type: CommitTypeV2::Force as i32, + subrequests: vec![PersistSubrequest { + subrequest_id: 0, + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(0)), + doc_batch: Some(DocBatchV2::for_test([ + "", + "[]", + r#"{"foo": "bar"}"#, + r#"{"doc": "test-doc-000"}"#, + ])), + }], + }; let persist_response = ingester.persist(persist_request).await.unwrap(); assert_eq!(persist_response.leader_id, "test-ingester"); assert_eq!(persist_response.successes.len(), 1); assert_eq!(persist_response.failures.len(), 0); let persist_success = &persist_response.successes[0]; - assert_eq!(persist_success.subrequest_id, 0); - assert_eq!(persist_success.index_uid(), &index_uid); - assert_eq!(persist_success.source_id, "test-source"); - assert_eq!(persist_success.shard_id(), ShardId::from(1)); - assert_eq!( - persist_success.replication_position_inclusive, - Some(Position::Beginning) - ); + assert_eq!(persist_success.num_persisted_docs, 1); + assert_eq!(persist_success.parse_failures.len(), 3); + + let parse_failure_0 = &persist_success.parse_failures[0]; + assert_eq!(parse_failure_0.doc_uid(), DocUid::for_test(0)); + assert_eq!(parse_failure_0.reason(), ParseFailureReason::InvalidJson); + assert!(parse_failure_0.message.contains("parse JSON document")); + + let parse_failure_1 = &persist_success.parse_failures[1]; + assert_eq!(parse_failure_1.doc_uid(), DocUid::for_test(1)); + assert_eq!(parse_failure_1.reason(), ParseFailureReason::InvalidJson); + assert!(parse_failure_1.message.contains("not an object")); + + let parse_failure_2 = &persist_success.parse_failures[2]; + assert_eq!(parse_failure_2.doc_uid(), DocUid::for_test(2)); + assert_eq!(parse_failure_2.reason(), ParseFailureReason::InvalidSchema); + assert!(parse_failure_2.message.contains("not declared")); } // This test should be run manually and independently of other tests with the `failpoints` @@ -1805,6 +1993,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, Instant::now(), ); state_guard.shards.insert(queue_id.clone(), solo_shard); @@ -1831,7 +2020,7 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), + doc_batch: Some(DocBatchV2::for_test([r#"test-doc-foo"#])), }], }; let persist_response = ingester.persist(persist_request).await.unwrap(); @@ -1858,12 +2047,18 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let mut state_guard = ingester.state.lock_fully().await.unwrap(); + let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + + let doc_mapper = try_build_doc_mapper("{}").unwrap(); + + // Insert a dangling shard, i.e. a shard without a corresponding queue. let solo_shard = IngesterShard::new_solo( ShardState::Open, Position::Beginning, Position::Beginning, + Some(doc_mapper), Instant::now(), ); state_guard.shards.insert(queue_id.clone(), solo_shard); @@ -1884,7 +2079,7 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-foo"}"#])), }], }; let persist_response = ingester.persist(persist_request).await.unwrap(); @@ -1930,6 +2125,12 @@ mod tests { let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let init_shards_request = InitShardsRequest { subrequests: vec![ InitShardSubrequest { @@ -1941,8 +2142,10 @@ mod tests { shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), follower_id: Some(follower_ctx.node_id.to_string()), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json: doc_mapping_json.clone(), }, InitShardSubrequest { subrequest_id: 1, @@ -1953,8 +2156,10 @@ mod tests { shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), follower_id: Some(follower_ctx.node_id.to_string()), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json, }, ], }; @@ -1969,14 +2174,17 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-110", "test-doc-111"])), + doc_batch: Some(DocBatchV2::for_test([ + r#"{"doc": "test-doc-110"}"#, + r#"{"doc": "test-doc-111"}"#, + ])), }, ], }; @@ -2017,7 +2225,7 @@ mod tests { leader_state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010"), (1, "\0\x01")], + &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#), (1, [0, 1], "")], ); let queue_id_11 = queue_id(&index_uid2, "test-source", &ShardId::from(1)); @@ -2030,9 +2238,9 @@ mod tests { &queue_id_11, .., &[ - (0, "\0\0test-doc-110"), - (1, "\0\0test-doc-111"), - (2, "\0\x01"), + (0, [0, 0], r#"{"doc": "test-doc-110"}"#), + (1, [0, 0], r#"{"doc": "test-doc-111"}"#), + (2, [0, 1], ""), ], ); @@ -2047,7 +2255,7 @@ mod tests { follower_state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010"), (1, "\0\x01")], + &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#), (1, [0, 1], "")], ); let replica_shard_11 = follower_state_guard.shards.get(&queue_id_11).unwrap(); @@ -2059,9 +2267,9 @@ mod tests { &queue_id_11, .., &[ - (0, "\0\0test-doc-110"), - (1, "\0\0test-doc-111"), - (2, "\0\x01"), + (0, [0, 0], r#"{"doc": "test-doc-110"}"#), + (1, [0, 0], r#"{"doc": "test-doc-111"}"#), + (2, [0, 1], ""), ], ); } @@ -2122,6 +2330,12 @@ mod tests { let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let init_shards_request = InitShardsRequest { subrequests: vec![ InitShardSubrequest { @@ -2133,8 +2347,10 @@ mod tests { shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), follower_id: Some(follower_ctx.node_id.to_string()), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json: doc_mapping_json.clone(), }, InitShardSubrequest { subrequest_id: 1, @@ -2145,8 +2361,10 @@ mod tests { shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), follower_id: Some(follower_ctx.node_id.to_string()), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }), + doc_mapping_json, }, ], }; @@ -2161,14 +2379,17 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-110", "test-doc-111"])), + doc_batch: Some(DocBatchV2::for_test([ + r#"{"doc": "test-doc-110"}"#, + r#"{"doc": "test-doc-111"}"#, + ])), }, ], }; @@ -2209,7 +2430,7 @@ mod tests { leader_state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010")], + &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#)], ); let queue_id_11 = queue_id(&index_uid2, "test-source", &ShardId::from(1)); @@ -2221,7 +2442,10 @@ mod tests { leader_state_guard.mrecordlog.assert_records_eq( &queue_id_11, .., - &[(0, "\0\0test-doc-110"), (1, "\0\0test-doc-111")], + &[ + (0, [0, 0], r#"{"doc": "test-doc-110"}"#), + (1, [0, 0], r#"{"doc": "test-doc-111"}"#), + ], ); let follower_state_guard = follower.state.lock_fully().await.unwrap(); @@ -2235,7 +2459,7 @@ mod tests { follower_state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010")], + &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#)], ); let replica_shard_11 = follower_state_guard.shards.get(&queue_id_11).unwrap(); @@ -2246,7 +2470,10 @@ mod tests { follower_state_guard.mrecordlog.assert_records_eq( &queue_id_11, .., - &[(0, "\0\0test-doc-110"), (1, "\0\0test-doc-111")], + &[ + (0, [0, 0], r#"{"doc": "test-doc-110"}"#), + (1, [0, 0], r#"{"doc": "test-doc-111"}"#), + ], ); } @@ -2259,6 +2486,7 @@ mod tests { ShardState::Closed, Position::Beginning, Position::Beginning, + None, Instant::now(), ); ingester @@ -2277,7 +2505,7 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], }; let persist_response = ingester.persist(persist_request).await.unwrap(); @@ -2312,22 +2540,31 @@ mod tests { .build() .await; - let mut state_guard = ingester.state.lock_fully().await.unwrap(); - let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let primary_shard = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; + let mut state_guard = ingester.state.lock_fully().await.unwrap(); + ingester .init_primary_shard( &mut state_guard.inner, &mut state_guard.mrecordlog, primary_shard, + &doc_mapping_json, Instant::now(), ) .await @@ -2343,7 +2580,7 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], }; let persist_response = ingester.persist(persist_request).await.unwrap(); @@ -2380,22 +2617,31 @@ mod tests { .build() .await; - let mut state_guard = ingester.state.lock_fully().await.unwrap(); - let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let primary_shard = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; + let mut state_guard = ingester.state.lock_fully().await.unwrap(); + ingester .init_primary_shard( &mut state_guard.inner, &mut state_guard.mrecordlog, primary_shard, + &doc_mapping_json, Instant::now(), ) .await @@ -2411,7 +2657,7 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], }; let persist_response = ingester.persist(persist_request).await.unwrap(); @@ -2497,11 +2743,18 @@ mod tests { matches!(error, IngestV2Error::ShardNotFound { shard_id } if shard_id == ShardId::from(1337)) ); + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); @@ -2513,6 +2766,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard, + &doc_mapping_json, Instant::now(), ) .await @@ -2596,24 +2850,38 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); + + let doc_mapping_uid_01 = DocMappingUid::random(); + let doc_mapping_json_01 = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid_01}" + }}"# + ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid_01), ..Default::default() }; - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let doc_mapping_uid_02 = DocMappingUid::random(); + let doc_mapping_json_02 = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid_02}" + }}"# + ); let shard_02 = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(doc_mapping_uid_02), ..Default::default() }; - let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); - let mut state_guard = ingester.state.lock_fully().await.unwrap(); let now = Instant::now(); @@ -2622,6 +2890,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_01, + &doc_mapping_json_01, now, ) .await @@ -2631,11 +2900,15 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_02, + &doc_mapping_json_02, now, ) .await .unwrap(); + assert_eq!(state_guard.shards.len(), 2); + assert_eq!(state_guard.doc_mappers.len(), 2); + let records = [ MRecord::new_doc("test-doc-foo").encode(), MRecord::new_doc("test-doc-bar").encode(), @@ -2688,21 +2961,21 @@ mod tests { // Verify idempotency. ingester - .truncate_shards(truncate_shards_request.clone()) + .truncate_shards(truncate_shards_request) .await .unwrap(); let state_guard = ingester.state.lock_fully().await.unwrap(); + assert_eq!(state_guard.shards.len(), 1); + assert_eq!(state_guard.doc_mappers.len(), 1); assert!(state_guard.shards.contains_key(&queue_id_01)); + assert!(state_guard.doc_mappers.contains_key(&doc_mapping_uid_01)); state_guard .mrecordlog - .assert_records_eq(&queue_id_01, .., &[(1, "\0\0test-doc-bar")]); - - assert!(!state_guard.shards.contains_key(&queue_id_02)); - assert!(!state_guard.mrecordlog.queue_exists(&queue_id_02)); + .assert_records_eq(&queue_id_01, .., &[(1, [0, 0], "test-doc-bar")]); } #[tokio::test] @@ -2717,6 +2990,7 @@ mod tests { ShardState::Open, Position::Beginning, Position::Beginning, + None, Instant::now(), ); state_guard.shards.insert(queue_id.clone(), solo_shard); @@ -2794,11 +3068,18 @@ mod tests { let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let shard_02 = Shard { @@ -2806,6 +3087,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); @@ -2818,6 +3100,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_01, + &doc_mapping_json, now, ) .await @@ -2827,6 +3110,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_02, + &doc_mapping_json, now, ) .await @@ -2860,11 +3144,19 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard_17 = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(17)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; @@ -2873,6 +3165,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(18)), shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let queue_id_17 = queue_id( @@ -2889,15 +3182,18 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_17, + &doc_mapping_json, now, ) .await .unwrap(); + ingester .init_primary_shard( &mut state_guard.inner, &mut state_guard.mrecordlog, shard_18, + &doc_mapping_json, now, ) .await @@ -2931,22 +3227,30 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), publish_position_inclusive: Some(Position::Beginning), ..Default::default() }; - let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); - let mut state_guard = ingester.state.lock_fully().await.unwrap(); ingester .init_primary_shard( &mut state_guard.inner, &mut state_guard.mrecordlog, shard, + &doc_mapping_json, Instant::now(), ) .await @@ -3059,6 +3363,7 @@ mod tests { ShardState::Closed, Position::offset(12u64), Position::Beginning, + None, Instant::now(), ), ); @@ -3079,11 +3384,19 @@ mod tests { ingester.subscribe(&event_broker); let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); @@ -3093,6 +3406,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); @@ -3105,6 +3419,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_01, + &doc_mapping_json, now, ) .await @@ -3114,6 +3429,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_02, + &doc_mapping_json, now, ) .await @@ -3167,7 +3483,7 @@ mod tests { state_guard .mrecordlog - .assert_records_eq(&queue_id_01, .., &[(1, "\0\0test-doc-bar")]); + .assert_records_eq(&queue_id_01, .., &[(1, [0, 0], "test-doc-bar")]); assert!(!state_guard.shards.contains_key(&queue_id_02)); assert!(!state_guard.mrecordlog.queue_exists(&queue_id_02)); @@ -3175,8 +3491,6 @@ mod tests { #[tokio::test] async fn test_ingester_closes_idle_shards() { - // The `CloseIdleShardsTask` task is already unit tested, so this test ensures the task is - // correctly spawned upon starting an ingester. let idle_shard_timeout = Duration::from_millis(200); let (_ingester_ctx, ingester) = IngesterForTest::default() .with_idle_shard_timeout(idle_shard_timeout) @@ -3185,12 +3499,28 @@ mod tests { let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), + ..Default::default() + }; + let shard_02 = Shard { + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(2)), + shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let mut state_guard = ingester.state.lock_fully().await.unwrap(); @@ -3201,21 +3531,37 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_01, + &doc_mapping_json, now - idle_shard_timeout, ) .await .unwrap(); - + ingester + .init_primary_shard( + &mut state_guard.inner, + &mut state_guard.mrecordlog, + shard_02, + &doc_mapping_json, + now, + ) + .await + .unwrap(); drop(state_guard); - tokio::time::sleep(Duration::from_millis(500)).await; - let state_guard = ingester.state.lock_partially().await.unwrap(); + tokio::time::sleep(Duration::from_millis(100)).await; // 2 times the run interval period of the close idle shards task + let state_guard = ingester.state.lock_partially().await.unwrap(); state_guard .shards .get(&queue_id_01) .unwrap() .assert_is_closed(); + state_guard + .shards + .get(&queue_id_02) + .unwrap() + .assert_is_open(); + drop(state_guard); } #[tokio::test] @@ -3223,11 +3569,20 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid_0: IndexUid = IndexUid::for_test("test-index-0", 0); + let index_uid_1: IndexUid = IndexUid::for_test("test-index-1", 0); + + let doc_mapping_uid = DocMappingUid::random(); + let doc_mapping_json = format!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); let shard_01 = Shard { index_uid: Some(index_uid_0.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; let shard_02 = Shard { @@ -3235,17 +3590,17 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; - let index_uid_1: IndexUid = IndexUid::for_test("test-index-1", 0); let shard_03 = Shard { index_uid: Some(index_uid_1.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(3)), shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; - let mut state_guard = ingester.state.lock_fully().await.unwrap(); let now = Instant::now(); @@ -3254,6 +3609,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_01, + &doc_mapping_json, now, ) .await @@ -3263,6 +3619,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_02, + &doc_mapping_json, now, ) .await @@ -3272,6 +3629,7 @@ mod tests { &mut state_guard.inner, &mut state_guard.mrecordlog, shard_03, + &doc_mapping_json, now, ) .await diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index 55c6a0aee9f..76d3881ec86 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -19,6 +19,7 @@ mod broadcast; mod debouncing; +mod doc_mapper; mod fetch; mod idle; mod ingester; @@ -46,7 +47,7 @@ use quickwit_common::tower::Pool; use quickwit_proto::ingest::ingester::IngesterServiceClient; use quickwit_proto::ingest::router::{IngestRequestV2, IngestSubrequest}; use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2}; -use quickwit_proto::types::{IndexId, NodeId}; +use quickwit_proto::types::{DocUid, DocUidGenerator, IndexId, NodeId, SubrequestId}; use tracing::{error, info}; pub use self::fetch::{FetchStreamError, MultiFetchStream}; @@ -112,23 +113,26 @@ pub(crate) fn get_ingest_router_buffer_size() -> ByteSize { /// Helper struct to build a [`DocBatchV2`]`. #[derive(Debug, Default)] pub struct DocBatchV2Builder { + doc_uids: Vec, doc_buffer: BytesMut, doc_lengths: Vec, } impl DocBatchV2Builder { /// Adds a document to the batch. - pub fn add_doc(&mut self, doc: &[u8]) { - self.doc_lengths.push(doc.len() as u32); + pub fn add_doc(&mut self, doc_uid: DocUid, doc: &[u8]) { + self.doc_uids.push(doc_uid); self.doc_buffer.put(doc); + self.doc_lengths.push(doc.len() as u32); } /// Builds the [`DocBatchV2`], returning `None` if the batch is empty. pub fn build(self) -> Option { - if self.doc_lengths.is_empty() { + if self.doc_uids.is_empty() { return None; } let doc_batch = DocBatchV2 { + doc_uids: self.doc_uids, doc_buffer: self.doc_buffer.freeze(), doc_lengths: self.doc_lengths, }; @@ -139,26 +143,30 @@ impl DocBatchV2Builder { /// Helper struct to build an [`IngestRequestV2`]. #[derive(Debug, Default)] pub struct IngestRequestV2Builder { - per_index_id_doc_batch_builders: HashMap, - subrequest_id_sequence: u32, + per_index_id_doc_batch_builders: HashMap, + subrequest_id_sequence: SubrequestId, + doc_uid_generator: DocUidGenerator, } impl IngestRequestV2Builder { - /// Adds a document to the request. - pub fn add_doc(&mut self, index_id: IndexId, doc: &[u8]) -> u32 { + /// Adds a document to the request, returning the ID of the subrequest to wich it was added and + /// its newly assigned [`DocUid`]. + pub fn add_doc(&mut self, index_id: IndexId, doc: &[u8]) -> (SubrequestId, DocUid) { match self.per_index_id_doc_batch_builders.entry(index_id) { Entry::Occupied(mut entry) => { let (subrequest_id, doc_batch_builder) = entry.get_mut(); - doc_batch_builder.add_doc(doc); - *subrequest_id + let doc_uid = self.doc_uid_generator.next_doc_uid(); + doc_batch_builder.add_doc(doc_uid, doc); + (*subrequest_id, doc_uid) } Entry::Vacant(entry) => { let subrequest_id = self.subrequest_id_sequence; self.subrequest_id_sequence += 1; let mut doc_batch_builder = DocBatchV2Builder::default(); - doc_batch_builder.add_doc(doc); + let doc_uid = self.doc_uid_generator.next_doc_uid(); + doc_batch_builder.add_doc(doc_uid, doc); entry.insert((subrequest_id, doc_batch_builder)); - subrequest_id + (subrequest_id, doc_uid) } } } @@ -240,8 +248,9 @@ mod tests { assert!(doc_batch_opt.is_none()); let mut doc_batch_builder = DocBatchV2Builder::default(); - doc_batch_builder.add_doc(b"Hello, "); - doc_batch_builder.add_doc(b"World!"); + let mut doc_uid_generator = DocUidGenerator::default(); + doc_batch_builder.add_doc(doc_uid_generator.next_doc_uid(), b"Hello, "); + doc_batch_builder.add_doc(doc_uid_generator.next_doc_uid(), b"World!"); let doc_batch = doc_batch_builder.build().unwrap(); assert_eq!(doc_batch.num_docs(), 2); @@ -257,11 +266,26 @@ mod tests { assert!(ingest_request_opt.is_none()); let mut ingest_request_builder = IngestRequestV2Builder::default(); - ingest_request_builder.add_doc("test-index-foo".to_string(), b"Hello, "); - ingest_request_builder.add_doc("test-index-foo".to_string(), b"World!"); - ingest_request_builder.add_doc("test-index-bar".to_string(), b"Hola, "); - ingest_request_builder.add_doc("test-index-bar".to_string(), b"Mundo!"); + let (subrequest_id, hello_doc_uid) = + ingest_request_builder.add_doc("test-index-foo".to_string(), b"Hello, "); + assert_eq!(subrequest_id, 0); + + let (subrequest_id, world_doc_uid) = + ingest_request_builder.add_doc("test-index-foo".to_string(), b"World!"); + assert_eq!(subrequest_id, 0); + assert!(hello_doc_uid < world_doc_uid); + + let (subrequest_id, hola_doc_uid) = + ingest_request_builder.add_doc("test-index-bar".to_string(), b"Hola, "); + assert_eq!(subrequest_id, 1); + assert!(world_doc_uid < hola_doc_uid); + + let (subrequest_id, mundo_doc_uid) = + ingest_request_builder.add_doc("test-index-bar".to_string(), b"Mundo!"); + assert_eq!(subrequest_id, 1); + assert!(hola_doc_uid < mundo_doc_uid); + let mut ingest_request = ingest_request_builder .build("test-source", CommitTypeV2::Auto) .unwrap(); @@ -305,6 +329,14 @@ mod tests { .doc_buffer, Bytes::from(&b"Hello, World!"[..]) ); + assert_eq!( + ingest_request.subrequests[0] + .doc_batch + .as_ref() + .unwrap() + .doc_uids, + [hello_doc_uid, world_doc_uid] + ); assert_eq!(ingest_request.subrequests[1].index_id, "test-index-bar"); assert_eq!(ingest_request.subrequests[1].source_id, "test-source"); @@ -340,6 +372,14 @@ mod tests { .doc_buffer, Bytes::from(&b"Hola, Mundo!"[..]) ); + assert_eq!( + ingest_request.subrequests[1] + .doc_batch + .as_ref() + .unwrap() + .doc_uids, + [hola_doc_uid, mundo_doc_uid] + ); } #[test] @@ -347,12 +387,14 @@ mod tests { let doc_batch = DocBatchV2 { doc_buffer: Vec::new().into(), doc_lengths: Vec::new(), + doc_uids: Vec::new(), }; assert_eq!(estimate_size(&doc_batch), ByteSize(0)); let doc_batch = DocBatchV2 { doc_buffer: vec![0u8; 100].into(), doc_lengths: vec![10, 20, 30], + doc_uids: Vec::new(), }; assert_eq!(estimate_size(&doc_batch), ByteSize(118)); } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/models.rs b/quickwit/quickwit-ingest/src/ingest_v2/models.rs index aee61f2130e..8d93954f9f9 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/models.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/models.rs @@ -17,13 +17,15 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::sync::Arc; use std::time::{Duration, Instant}; +use quickwit_doc_mapper::DocMapper; use quickwit_proto::ingest::ShardState; use quickwit_proto::types::{NodeId, Position}; use tokio::sync::watch; -#[derive(Debug, Clone, Eq, PartialEq)] +#[derive(Debug, Clone)] pub(super) enum IngesterShardType { /// A primary shard hosted on a leader and replicated on a follower. Primary { follower_id: NodeId }, @@ -51,6 +53,8 @@ pub(super) struct IngesterShard { /// was successfully opened before advertising it. Currently, this confirmation comes in the /// form of `PersistRequest` or `FetchRequest`. pub is_advertisable: bool, + /// Document mapper for the shard. Replica shards and closed solo shards do not have one. + pub doc_mapper_opt: Option>, pub shard_status_tx: watch::Sender, pub shard_status_rx: watch::Receiver, /// Instant at which the shard was last written to. @@ -63,6 +67,7 @@ impl IngesterShard { shard_state: ShardState, replication_position_inclusive: Position, truncation_position_inclusive: Position, + doc_mapper: Arc, now: Instant, ) -> Self { let shard_status = (shard_state, replication_position_inclusive.clone()); @@ -73,6 +78,7 @@ impl IngesterShard { replication_position_inclusive, truncation_position_inclusive, is_advertisable: false, + doc_mapper_opt: Some(doc_mapper), shard_status_tx, shard_status_rx, last_write_instant: now, @@ -96,6 +102,7 @@ impl IngesterShard { // This is irrelevant for replica shards since they are not advertised via gossip // anyway. is_advertisable: false, + doc_mapper_opt: None, shard_status_tx, shard_status_rx, last_write_instant: now, @@ -106,6 +113,7 @@ impl IngesterShard { shard_state: ShardState, replication_position_inclusive: Position, truncation_position_inclusive: Position, + doc_mapper_opt: Option>, now: Instant, ) -> Self { let shard_status = (shard_state, replication_position_inclusive.clone()); @@ -116,6 +124,7 @@ impl IngesterShard { replication_position_inclusive, truncation_position_inclusive, is_advertisable: false, + doc_mapper_opt, shard_status_tx, shard_status_rx, last_write_instant: now, @@ -124,7 +133,7 @@ impl IngesterShard { pub fn follower_id_opt(&self) -> Option<&NodeId> { match &self.shard_type { - IngesterShardType::Primary { follower_id } => Some(follower_id), + IngesterShardType::Primary { follower_id, .. } => Some(follower_id), IngesterShardType::Replica { .. } => None, IngesterShardType::Solo => None, } @@ -182,6 +191,8 @@ impl IngesterShard { #[cfg(test)] mod tests { + use quickwit_config::{build_doc_mapper, DocMapping, SearchSettings}; + use super::*; impl IngesterShard { @@ -231,16 +242,21 @@ mod tests { #[test] fn test_new_primary_shard() { + let doc_mapping: DocMapping = serde_json::from_str("{}").unwrap(); + let search_settings = SearchSettings::default(); + let doc_mapper = build_doc_mapper(&doc_mapping, &search_settings).unwrap(); + let primary_shard = IngesterShard::new_primary( "test-follower".into(), ShardState::Closed, Position::offset(42u64), Position::Beginning, + doc_mapper, Instant::now(), ); assert!(matches!( &primary_shard.shard_type, - IngesterShardType::Primary { follower_id } if *follower_id == "test-follower" + IngesterShardType::Primary { follower_id, .. } if *follower_id == "test-follower" )); assert!(!primary_shard.is_replica()); assert_eq!(primary_shard.shard_state, ShardState::Closed); @@ -287,9 +303,10 @@ mod tests { ShardState::Closed, Position::offset(42u64), Position::Beginning, + None, Instant::now(), ); - assert_eq!(solo_shard.shard_type, IngesterShardType::Solo); + solo_shard.assert_is_solo(); assert!(!solo_shard.is_replica()); assert_eq!(solo_shard.shard_state, ShardState::Closed); assert_eq!( diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs b/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs index 8ca72a420a0..b9b7e6c11d4 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs @@ -52,8 +52,8 @@ pub(super) async fn append_non_empty_doc_batch( ) -> Result { let append_result = if force_commit { let encoded_mrecords = doc_batch - .docs() - .map(|doc| MRecord::Doc(doc).encode()) + .into_docs() + .map(|(_doc_uid, doc)| MRecord::Doc(doc).encode()) .chain(once(MRecord::Commit.encode())); #[cfg(feature = "failpoints")] @@ -66,7 +66,9 @@ pub(super) async fn append_non_empty_doc_batch( .append_records(queue_id, None, encoded_mrecords) .await } else { - let encoded_mrecords = doc_batch.docs().map(|doc| MRecord::Doc(doc).encode()); + let encoded_mrecords = doc_batch + .into_docs() + .map(|(_doc_uid, doc)| MRecord::Doc(doc).encode()); #[cfg(feature = "failpoints")] fail_point!("ingester:append_records", |_| { diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 06c6a5b688b..8bbe5f732f3 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -1239,18 +1239,18 @@ mod tests { state_guard .mrecordlog - .assert_records_eq(&queue_id_01, .., &[(0, "\0\0test-doc-foo")]); + .assert_records_eq(&queue_id_01, .., &[(0, [0, 0], "test-doc-foo")]); state_guard.mrecordlog.assert_records_eq( &queue_id_02, .., - &[(0, "\0\0test-doc-bar"), (1, "\0\0test-doc-baz")], + &[(0, [0, 0], "test-doc-bar"), (1, [0, 0], "test-doc-baz")], ); state_guard.mrecordlog.assert_records_eq( &queue_id_11, .., - &[(0, "\0\0test-doc-qux"), (1, "\0\0test-doc-tux")], + &[(0, [0, 0], "test-doc-qux"), (1, [0, 0], "test-doc-tux")], ); drop(state_guard); @@ -1296,7 +1296,7 @@ mod tests { state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-foo"), (1, "\0\0test-doc-moo")], + &[(0, [0, 0], "test-doc-foo"), (1, [0, 0], "test-doc-moo")], ); } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index fefdffb2834..4eaf7803efb 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -189,12 +189,10 @@ impl IngestRouter { drop(state_guard); if !debounced_request.is_empty() && !debounced_request.closed_shards.is_empty() { - info!(closed_shards=?debounced_request.closed_shards, "reporting closed shard(s) to - control plane"); + info!(closed_shards=?debounced_request.closed_shards, "reporting closed shard(s) to control plane"); } if !debounced_request.is_empty() && !unavailable_leaders.is_empty() { - info!(unvailable_leaders=?unavailable_leaders, "reporting unavailable leader(s) to - control plane"); + info!(unvailable_leaders=?unavailable_leaders, "reporting unavailable leader(s) to control plane"); for unavailable_leader in unavailable_leaders.iter() { debounced_request @@ -564,8 +562,10 @@ mod tests { IngesterServiceClient, MockIngesterService, PersistFailure, PersistResponse, PersistSuccess, }; use quickwit_proto::ingest::router::IngestSubrequest; - use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2, Shard, ShardIds, ShardState}; - use quickwit_proto::types::{Position, SourceUid}; + use quickwit_proto::ingest::{ + CommitTypeV2, DocBatchV2, ParseFailure, ParseFailureReason, Shard, ShardIds, ShardState, + }; + use quickwit_proto::types::{DocUid, Position, SourceUid}; use tokio::task::yield_now; use super::*; @@ -1350,7 +1350,7 @@ mod tests { assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo", "test-doc-bar"])) + Some(DocBatchV2::for_test(["", "test-doc-foo", "test-doc-bar"])) ); let subrequest = &request.subrequests[1]; @@ -1372,6 +1372,12 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(1u64)), + num_persisted_docs: 2, + parse_failures: vec![ParseFailure { + doc_uid: Some(DocUid::for_test(0)), + reason: ParseFailureReason::InvalidJson as i32, + message: "invalid JSON".to_string(), + }], }, PersistSuccess { subrequest_id: 1, @@ -1379,6 +1385,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), + num_persisted_docs: 1, + parse_failures: Vec::new(), }, ], failures: Vec::new(), @@ -1411,6 +1419,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(3u64)), + num_persisted_docs: 4, + parse_failures: Vec::new(), }], failures: Vec::new(), }; @@ -1446,6 +1456,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(2)), replication_position_inclusive: Some(Position::offset(0u64)), + num_persisted_docs: 1, + parse_failures: Vec::new(), }], failures: Vec::new(), }; @@ -1460,7 +1472,7 @@ mod tests { subrequest_id: 0, index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["test-doc-foo", "test-doc-bar"])), + doc_batch: Some(DocBatchV2::for_test(["", "test-doc-foo", "test-doc-bar"])), }, IngestSubrequest { subrequest_id: 1, @@ -1471,7 +1483,16 @@ mod tests { ], commit_type: CommitTypeV2::Auto as i32, }; - router.ingest(ingest_request).await.unwrap(); + let response = router.ingest(ingest_request).await.unwrap(); + assert_eq!(response.successes.len(), 2); + assert_eq!(response.failures.len(), 0); + + let parse_failures = &response.successes[0].parse_failures; + assert_eq!(parse_failures.len(), 1); + + let parse_failure = &parse_failures[0]; + assert_eq!(parse_failure.doc_uid(), DocUid::for_test(0)); + assert_eq!(parse_failure.reason(), ParseFailureReason::InvalidJson); let ingest_request = IngestRequestV2 { subrequests: vec![ @@ -1490,7 +1511,9 @@ mod tests { ], commit_type: CommitTypeV2::Auto as i32, }; - router.ingest(ingest_request).await.unwrap(); + let response = router.ingest(ingest_request).await.unwrap(); + assert_eq!(response.successes.len(), 2); + assert_eq!(response.failures.len(), 0); } #[tokio::test] @@ -1580,6 +1603,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), + num_persisted_docs: 1, + parse_failures: Vec::new(), }], failures: Vec::new(), }; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs index a0351bac59d..7c4ed6f7fed 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs @@ -234,7 +234,7 @@ impl RoutingTableEntry { target_shards.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); info!( - index_id=%self.index_uid.index_id, + index_uid=%self.index_uid, source_id=%self.source_id, "inserted {num_inserted_shards} shards into routing table" ); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/state.rs b/quickwit/quickwit-ingest/src/ingest_v2/state.rs index 8cbfe81f5f4..050155919d7 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/state.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/state.rs @@ -27,10 +27,11 @@ use std::time::{Duration, Instant}; use mrecordlog::error::{DeleteQueueError, TruncateError}; use quickwit_common::pretty::PrettyDisplay; use quickwit_common::rate_limiter::{RateLimiter, RateLimiterSettings}; +use quickwit_doc_mapper::DocMapper; use quickwit_proto::control_plane::AdviseResetShardsResponse; use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::ingest::{IngestV2Error, IngestV2Result, ShardState}; -use quickwit_proto::types::{Position, QueueId}; +use quickwit_proto::types::{DocMappingUid, Position, QueueId}; use tokio::sync::{watch, Mutex, MutexGuard, RwLock, RwLockMappedWriteGuard, RwLockWriteGuard}; use tracing::{error, info}; @@ -57,6 +58,7 @@ pub(super) struct IngesterState { pub(super) struct InnerIngesterState { pub shards: HashMap, + pub doc_mappers: HashMap>, pub rate_trackers: HashMap, // Replication stream opened with followers. pub replication_streams: HashMap, @@ -83,6 +85,7 @@ impl IngesterState { let (status_tx, status_rx) = watch::channel(status); let inner = InnerIngesterState { shards: Default::default(), + doc_mappers: Default::default(), rate_trackers: Default::default(), replication_streams: Default::default(), replication_tasks: Default::default(), @@ -185,6 +188,7 @@ impl IngesterState { ShardState::Closed, replication_position_inclusive, truncation_position_inclusive, + None, now, ); // We want to advertise the shard as read-only right away. @@ -342,8 +346,21 @@ impl FullyLockedIngesterState<'_> { self.rate_trackers.remove(queue_id); // Log only if the shard was actually removed. - if self.shards.remove(queue_id).is_some() { + if let Some(shard) = self.shards.remove(queue_id) { info!("deleted shard `{queue_id}`"); + + if let Some(doc_mapper) = shard.doc_mapper_opt { + // At this point, we hold the lock so we can safely check the strong count. + // The other locations where the doc mapper is cloned also require holding + // the lock. + if Arc::strong_count(&doc_mapper) == 1 { + let doc_mapping_uid = doc_mapper.doc_mapping_uid(); + + if self.doc_mappers.remove(&doc_mapping_uid).is_some() { + info!("evicted doc mapper `{doc_mapping_uid}` from cache`"); + } + } + } } } Err(DeleteQueueError::IoError(io_error)) => { @@ -394,7 +411,7 @@ impl FullyLockedIngesterState<'_> { /// Deletes and truncates the shards as directed by the `advise_reset_shards_response` returned /// by the control plane. pub async fn reset_shards(&mut self, advise_reset_shards_response: &AdviseResetShardsResponse) { - info!("reset shards"); + info!("resetting shards"); for shard_ids in &advise_reset_shards_response.shards_to_delete { for queue_id in shard_ids.queue_ids() { self.delete_shard(&queue_id).await; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index b6e06f9ef5d..501dedf99dd 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -212,6 +212,8 @@ impl IngestWorkbench { source_id: persist_success.source_id, shard_id: persist_success.shard_id, replication_position_inclusive: persist_success.replication_position_inclusive, + num_ingested_docs: persist_success.num_persisted_docs, + parse_failures: persist_success.parse_failures, }; successes.push(success); } else if let Some(failure) = subworkbench.last_failure_opt { @@ -228,6 +230,16 @@ impl IngestWorkbench { let num_failures = failures.len(); assert_eq!(num_successes + num_failures, num_subworkbenches); + #[cfg(test)] + { + for success in &mut successes { + success + .parse_failures + .sort_by_key(|parse_failure| parse_failure.doc_uid()); + } + successes.sort_by_key(|success| success.subrequest_id); + failures.sort_by_key(|failure| failure.subrequest_id); + } if self.num_successes == 0 && num_failures > 0 && failures.iter().all(|failure| { diff --git a/quickwit/quickwit-ingest/src/mrecordlog_async.rs b/quickwit/quickwit-ingest/src/mrecordlog_async.rs index 0c331da2548..fb3c598551e 100644 --- a/quickwit/quickwit-ingest/src/mrecordlog_async.rs +++ b/quickwit/quickwit-ingest/src/mrecordlog_async.rs @@ -126,13 +126,21 @@ impl MultiRecordLogAsync { #[track_caller] #[cfg(test)] - pub fn assert_records_eq(&self, queue_id: &str, range: R, expected_records: &[(u64, &str)]) - where R: RangeBounds + 'static { + pub fn assert_records_eq( + &self, + queue_id: &str, + range: R, + expected_records: &[(u64, [u8; 2], &str)], + ) where + R: RangeBounds + 'static, + { let records = self .range(queue_id, range) .unwrap() .map(|Record { position, payload }| { - (position, String::from_utf8(payload.into_owned()).unwrap()) + let header: [u8; 2] = payload[..2].try_into().unwrap(); + let payload = String::from_utf8(payload[2..].to_vec()).unwrap(); + (position, header, payload) }) .collect::>(); assert_eq!( @@ -142,7 +150,7 @@ impl MultiRecordLogAsync { expected_records.len(), records.len() ); - for ((position, record), (expected_position, expected_record)) in + for ((position, header, payload), (expected_position, expected_header, expected_payload)) in records.iter().zip(expected_records.iter()) { assert_eq!( @@ -150,8 +158,12 @@ impl MultiRecordLogAsync { "expected record at position `{expected_position}`, got `{position}`", ); assert_eq!( - record, expected_record, - "expected record `{expected_record}`, got `{record}`", + header, expected_header, + "expected record header, `{expected_header:?}`, got `{header:?}`", + ); + assert_eq!( + payload, expected_payload, + "expected record payload, `{expected_payload}`, got `{payload}`", ); } } diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index 4108887fcb9..6d17ae0ebcf 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -63,12 +63,12 @@ pub(crate) use split_metadata_version::{SplitMetadataV0_8, VersionedSplitMetadat #[derive(utoipa::OpenApi)] #[openapi(components(schemas( + IndexMetadataV0_8, Split, + SplitMetadataV0_8, SplitState, VersionedIndexMetadata, - IndexMetadataV0_8, VersionedSplitMetadata, - SplitMetadataV0_8, )))] /// Schema used for the OpenAPI generation which are apart of this crate. pub struct MetastoreApiSchemas; diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index c4e2a3a3de4..ca28372a8f5 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -102,8 +102,8 @@ fn main() -> Result<(), Box> { ) .extern_path(".quickwit.common.IndexUid", "crate::types::IndexUid") .extern_path(".quickwit.ingest.ShardId", "crate::types::ShardId") - .field_attribute("DeleteQuery.index_uid", "#[serde(alias = \"index_id\")]") .field_attribute("DeleteQuery.index_uid", "#[schema(value_type = String)]") + .field_attribute("DeleteQuery.index_uid", "#[serde(alias = \"index_id\")]") .field_attribute("DeleteQuery.query_ast", "#[serde(alias = \"query\")]") .field_attribute( "DeleteQuery.start_timestamp", @@ -138,6 +138,7 @@ fn main() -> Result<(), Box> { ".quickwit.common.DocMappingUid", "crate::types::DocMappingUid", ) + .extern_path(".quickwit.common.DocUid", "crate::types::DocUid") .extern_path(".quickwit.common.IndexUid", "crate::types::IndexUid") .extern_path(".quickwit.ingest.Position", "crate::types::Position") .extern_path(".quickwit.ingest.ShardId", "crate::types::ShardId") diff --git a/quickwit/quickwit-proto/protos/quickwit/common.proto b/quickwit/quickwit-proto/protos/quickwit/common.proto index c7782e77f1e..73a6cbcec61 100644 --- a/quickwit/quickwit-proto/protos/quickwit/common.proto +++ b/quickwit/quickwit-proto/protos/quickwit/common.proto @@ -30,11 +30,21 @@ message DocMappingUid { bytes doc_mapping_uid = 1; } +// The corresponding Rust struct [`crate::types::DocUid`] is defined manually and +// externally provided during code generation (see `build.rs`). +// +// Modify at your own risk. +message DocUid { + // ULID encoded as a sequence of 16 bytes (big-endian u128). + bytes doc_uid = 1; +} + // The corresponding Rust struct [`crate::types::IndexUid`] is defined manually and // externally provided during code generation (see `build.rs`). // // Modify at your own risk. message IndexUid { string index_id = 1; + // ULID encoded as a sequence of 16 bytes (big-endian u128). bytes incarnation_id = 2; } diff --git a/quickwit/quickwit-proto/protos/quickwit/ingest.proto b/quickwit/quickwit-proto/protos/quickwit/ingest.proto index f4d24cace5d..fd546646bae 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingest.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingest.proto @@ -56,6 +56,7 @@ enum CommitTypeV2 { message DocBatchV2 { bytes doc_buffer = 1; repeated uint32 doc_lengths = 2; + repeated quickwit.common.DocUid doc_uids = 3; } message MRecordBatch { @@ -116,3 +117,15 @@ message ShardIdPosition { ShardId shard_id = 1; Position publish_position_inclusive = 2; } + +enum ParseFailureReason { + PARSE_FAILURE_REASON_UNSPECIFIED = 0; + PARSE_FAILURE_REASON_INVALID_JSON = 1; + PARSE_FAILURE_REASON_INVALID_SCHEMA = 2; +} + +message ParseFailure { + quickwit.common.DocUid doc_uid = 1; + ParseFailureReason reason = 2; + string message = 3; +} diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index 1ec77641945..45bc1d6c664 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -94,6 +94,8 @@ message PersistSuccess { string source_id = 3; quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.Position replication_position_inclusive = 5; + uint32 num_persisted_docs = 6; + repeated quickwit.ingest.ParseFailure parse_failures = 7; } enum PersistFailureReason { @@ -256,6 +258,7 @@ message InitShardsRequest { message InitShardSubrequest { uint32 subrequest_id = 1; quickwit.ingest.Shard shard = 2; + string doc_mapping_json = 3; } message InitShardsResponse { diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index fe7df81f6c4..c542f586787 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -348,13 +348,12 @@ message DeleteTask { message DeleteQuery { reserved 4, 5; - // Index ID. + // Index UID. quickwit.common.IndexUid index_uid = 1; // If set, restrict search to documents with a `timestamp >= start_timestamp`. optional int64 start_timestamp = 2; // If set, restrict search to documents with a `timestamp < end_timestamp``. optional int64 end_timestamp = 3; - // Query text. The query language is that of tantivy. // Query AST serialized in JSON string query_ast = 6; } diff --git a/quickwit/quickwit-proto/protos/quickwit/router.proto b/quickwit/quickwit-proto/protos/quickwit/router.proto index 2aed3550026..51677cfccb3 100644 --- a/quickwit/quickwit-proto/protos/quickwit/router.proto +++ b/quickwit/quickwit-proto/protos/quickwit/router.proto @@ -57,6 +57,8 @@ message IngestSuccess { quickwit.ingest.ShardId shard_id = 4; // Replication position inclusive. quickwit.ingest.Position replication_position_inclusive = 5; + uint32 num_ingested_docs = 6; + repeated quickwit.ingest.ParseFailure parse_failures = 7; } enum IngestFailureReason { diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.common.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.common.rs index 04e2e3f2c3b..d33e5dc658c 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.common.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.common.rs @@ -1,12 +1,12 @@ -/// The corresponding Rust struct \[`crate::types::DocMappingUid`\] is defined manually and +/// The corresponding Rust struct \[`crate::types::DocUid`\] is defined manually and /// externally provided during code generation (see `build.rs`). /// /// Modify at your own risk. #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct DocMappingUid { +pub struct DocUid { /// ULID encoded as a sequence of 16 bytes (big-endian u128). #[prost(bytes = "vec", tag = "1")] - pub doc_mapping_uid: ::prost::alloc::vec::Vec, + pub doc_uid: ::prost::alloc::vec::Vec, } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs index ace0b1d63b0..d23968d407e 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -71,6 +71,10 @@ pub struct PersistSuccess { pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, + #[prost(uint32, tag = "6")] + pub num_persisted_docs: u32, + #[prost(message, repeated, tag = "7")] + pub parse_failures: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -354,6 +358,8 @@ pub struct InitShardSubrequest { pub subrequest_id: u32, #[prost(message, optional, tag = "2")] pub shard: ::core::option::Option, + #[prost(string, tag = "3")] + pub doc_mapping_json: ::prost::alloc::string::String, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index dbc0f389539..dabfc978b63 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -47,6 +47,10 @@ pub struct IngestSuccess { /// Replication position inclusive. #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, + #[prost(uint32, tag = "6")] + pub num_ingested_docs: u32, + #[prost(message, repeated, tag = "7")] + pub parse_failures: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs index 97592b90e5f..c6efc801307 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -18,6 +18,8 @@ pub struct DocBatchV2 { pub doc_buffer: ::prost::bytes::Bytes, #[prost(uint32, repeated, tag = "2")] pub doc_lengths: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub doc_uids: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -99,6 +101,17 @@ pub struct ShardIdPosition { pub publish_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ParseFailure { + #[prost(message, optional, tag = "1")] + pub doc_uid: ::core::option::Option, + #[prost(enumeration = "ParseFailureReason", tag = "2")] + pub reason: i32, + #[prost(string, tag = "3")] + pub message: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] @@ -170,3 +183,34 @@ impl ShardState { } } } +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ParseFailureReason { + Unspecified = 0, + InvalidJson = 1, + InvalidSchema = 2, +} +impl ParseFailureReason { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + ParseFailureReason::Unspecified => "PARSE_FAILURE_REASON_UNSPECIFIED", + ParseFailureReason::InvalidJson => "PARSE_FAILURE_REASON_INVALID_JSON", + ParseFailureReason::InvalidSchema => "PARSE_FAILURE_REASON_INVALID_SCHEMA", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "PARSE_FAILURE_REASON_UNSPECIFIED" => Some(Self::Unspecified), + "PARSE_FAILURE_REASON_INVALID_JSON" => Some(Self::InvalidJson), + "PARSE_FAILURE_REASON_INVALID_SCHEMA" => Some(Self::InvalidSchema), + _ => None, + } + } +} diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index cd057f12a2b..191152a91eb 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -237,10 +237,10 @@ pub struct DeleteTask { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteQuery { - /// Index ID. + /// Index UID. #[prost(message, optional, tag = "1")] - #[serde(alias = "index_id")] #[schema(value_type = String)] + #[serde(alias = "index_id")] pub index_uid: ::core::option::Option, /// If set, restrict search to documents with a `timestamp >= start_timestamp`. #[prost(int64, optional, tag = "2")] @@ -250,7 +250,6 @@ pub struct DeleteQuery { #[prost(int64, optional, tag = "3")] #[serde(skip_serializing_if = "Option::is_none")] pub end_timestamp: ::core::option::Option, - /// Query text. The query language is that of tantivy. /// Query AST serialized in JSON #[prost(string, tag = "6")] #[serde(alias = "query")] diff --git a/quickwit/quickwit-proto/src/getters.rs b/quickwit/quickwit-proto/src/getters.rs index 0ab50d89fe2..9f8df0289ac 100644 --- a/quickwit/quickwit-proto/src/getters.rs +++ b/quickwit/quickwit-proto/src/getters.rs @@ -1,3 +1,22 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + use crate::control_plane::*; use crate::indexing::*; use crate::ingest::ingester::*; @@ -64,6 +83,13 @@ generate_copy_getters!( Shard ); +// [`DocUid`] getters +generate_copy_getters! { + impl fn doc_uid() -> DocUid {} for + + ParseFailure +} + // [`IndexUid`] getters generate_getters! { impl fn index_uid() -> &IndexUid {} for diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index 0369c207ab0..a4da613f300 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -17,6 +17,8 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::iter::zip; + use bytes::Bytes; use bytesize::ByteSize; use quickwit_common::tower::MakeLoadShedError; @@ -25,7 +27,7 @@ use self::ingester::{PersistFailureReason, ReplicateFailureReason}; use self::router::IngestFailureReason; use super::types::NodeId; use super::GrpcServiceError; -use crate::types::{queue_id, Position, QueueId, ShardId, SourceUid}; +use crate::types::{queue_id, DocUid, Position, QueueId, ShardId, SourceUid}; use crate::{ServiceError, ServiceErrorCode}; pub mod ingester; @@ -116,19 +118,24 @@ impl ShardPKey { } impl DocBatchV2 { - pub fn docs(self) -> impl Iterator { - let DocBatchV2 { - doc_buffer, - doc_lengths, - } = self; - doc_lengths - .into_iter() - .scan(0, move |start_offset, doc_length| { - let start = *start_offset; - let end = start + doc_length as usize; - *start_offset = end; - Some(doc_buffer.slice(start..end)) - }) + pub fn docs(&self) -> impl Iterator + '_ { + zip(&self.doc_uids, &self.doc_lengths).scan( + self.doc_buffer.clone(), + |doc_buffer, (doc_uid, doc_len)| { + let doc = doc_buffer.split_to(*doc_len as usize); + Some((*doc_uid, doc)) + }, + ) + } + + pub fn into_docs(self) -> impl Iterator { + zip(self.doc_uids, self.doc_lengths).scan( + self.doc_buffer, + |doc_buffer, (doc_uid, doc_len)| { + let doc = doc_buffer.split_to(doc_len as usize); + Some((doc_uid, doc)) + }, + ) } pub fn is_empty(&self) -> bool { @@ -145,16 +152,19 @@ impl DocBatchV2 { #[cfg(any(test, feature = "testsuite"))] pub fn for_test(docs: impl IntoIterator) -> Self { + let mut doc_uids = Vec::new(); let mut doc_buffer = Vec::new(); let mut doc_lengths = Vec::new(); - for doc in docs { + for (doc_uid, doc) in docs.into_iter().enumerate() { + doc_uids.push(DocUid::for_test(doc_uid as u128)); doc_buffer.extend(doc.as_bytes()); doc_lengths.push(doc.len() as u32); } Self { - doc_lengths, + doc_uids, doc_buffer: Bytes::from(doc_buffer), + doc_lengths, } } } diff --git a/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs b/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs index cbe6e873749..7f2690067cc 100644 --- a/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs +++ b/quickwit/quickwit-proto/src/types/doc_mapping_uid.rs @@ -26,7 +26,7 @@ use serde::de::Error; use serde::{Deserialize, Deserializer, Serialize, Serializer}; pub use ulid::Ulid; -use crate::types::pipeline_uid::ULID_SIZE; +use super::ULID_SIZE; /// Unique identifier for a document mapping. #[derive(Clone, Copy, Default, Hash, Eq, PartialEq, Ord, PartialOrd, utoipa::ToSchema)] diff --git a/quickwit/quickwit-proto/src/types/doc_uid.rs b/quickwit/quickwit-proto/src/types/doc_uid.rs new file mode 100644 index 00000000000..75e7b871e41 --- /dev/null +++ b/quickwit/quickwit-proto/src/types/doc_uid.rs @@ -0,0 +1,197 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::borrow::Cow; +use std::fmt; + +use serde::de::Error; +use serde::{Deserialize, Deserializer, Serialize, Serializer}; +pub use ulid::Ulid; + +use super::ULID_SIZE; + +/// A doc UID identifies a document across segments, splits, and indexes. +#[derive(Clone, Copy, Default, Hash, Eq, PartialEq, Ord, PartialOrd)] +pub struct DocUid(Ulid); + +impl fmt::Debug for DocUid { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "Doc({})", self.0) + } +} + +impl fmt::Display for DocUid { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + self.0.fmt(f) + } +} + +impl From for DocUid { + fn from(ulid: Ulid) -> Self { + Self(ulid) + } +} + +impl DocUid { + /// Creates a new random doc UID. + pub fn random() -> Self { + Self(Ulid::new()) + } + + #[cfg(any(test, feature = "testsuite"))] + pub fn for_test(ulid_u128: u128) -> DocUid { + Self(Ulid::from(ulid_u128)) + } +} + +impl<'de> Deserialize<'de> for DocUid { + fn deserialize(deserializer: D) -> Result + where D: Deserializer<'de> { + let doc_uid_str: Cow<'de, str> = Cow::deserialize(deserializer)?; + let doc_uid = Ulid::from_string(&doc_uid_str).map_err(D::Error::custom)?; + Ok(Self(doc_uid)) + } +} + +impl Serialize for DocUid { + fn serialize(&self, serializer: S) -> Result + where S: Serializer { + serializer.collect_str(&self.0) + } +} + +impl prost::Message for DocUid { + fn encode_raw(&self, buf: &mut B) + where B: prost::bytes::BufMut { + // TODO: when `bytes::encode` supports `&[u8]`, we can remove this allocation. + prost::encoding::bytes::encode(1u32, &self.0.to_bytes().to_vec(), buf); + } + + fn merge_field( + &mut self, + tag: u32, + wire_type: prost::encoding::WireType, + buf: &mut B, + ctx: prost::encoding::DecodeContext, + ) -> ::core::result::Result<(), prost::DecodeError> + where + B: prost::bytes::Buf, + { + const STRUCT_NAME: &str = "DocUid"; + + match tag { + 1u32 => { + let mut buffer = Vec::with_capacity(ULID_SIZE); + + prost::encoding::bytes::merge(wire_type, &mut buffer, buf, ctx).map_err( + |mut error| { + error.push(STRUCT_NAME, "doc_uid"); + error + }, + )?; + let ulid_bytes: [u8; ULID_SIZE] = + buffer.try_into().map_err(|buffer: Vec| { + prost::DecodeError::new(format!( + "invalid length for field `doc_uid`, expected 16 bytes, got {}", + buffer.len() + )) + })?; + self.0 = Ulid::from_bytes(ulid_bytes); + Ok(()) + } + _ => prost::encoding::skip_field(wire_type, tag, buf, ctx), + } + } + + #[inline] + fn encoded_len(&self) -> usize { + prost::encoding::key_len(1u32) + + prost::encoding::encoded_len_varint(ULID_SIZE as u64) + + ULID_SIZE + } + + fn clear(&mut self) { + self.0 = Ulid::nil(); + } +} + +/// Generates monotonically increasing doc UIDs. It is not `Clone` nor `Copy` on purpose. +#[derive(Debug)] +pub struct DocUidGenerator { + next_ulid: Ulid, +} + +impl Default for DocUidGenerator { + fn default() -> Self { + Self { + next_ulid: Ulid::new(), + } + } +} + +impl DocUidGenerator { + /// Generates a new doc UID. + #[allow(clippy::unwrap_or_default)] + pub fn next_doc_uid(&mut self) -> DocUid { + let doc_uid = DocUid(self.next_ulid); + // Clippy insists on using `unwrap_or_default`, but that's really not what we want here: + // https://github.com/rust-lang/rust-clippy/issues/11631 + self.next_ulid = self.next_ulid.increment().unwrap_or_else(Ulid::new); + doc_uid + } +} + +#[cfg(test)] +mod tests { + use bytes::Bytes; + use prost::Message; + + use super::*; + + #[test] + fn test_doc_uid_json_serde_roundtrip() { + let doc_uid = DocUid::default(); + let serialized = serde_json::to_string(&doc_uid).unwrap(); + assert_eq!(serialized, r#""00000000000000000000000000""#); + + let deserialized: DocUid = serde_json::from_str(&serialized).unwrap(); + assert_eq!(deserialized, doc_uid); + } + + #[test] + fn test_doc_uid_prost_serde_roundtrip() { + let doc_uid = DocUid::random(); + + let encoded = doc_uid.encode_to_vec(); + assert_eq!(DocUid::decode(Bytes::from(encoded)).unwrap(), doc_uid); + + let encoded = doc_uid.encode_length_delimited_to_vec(); + assert_eq!( + DocUid::decode_length_delimited(Bytes::from(encoded)).unwrap(), + doc_uid + ); + } + + #[test] + fn test_doc_uid_generator() { + let mut generator = DocUidGenerator::default(); + let doc_uids: Vec = (0..10_000).map(|_| generator.next_doc_uid()).collect(); + assert!(doc_uids.windows(2).all(|window| window[0] < window[1])); + } +} diff --git a/quickwit/quickwit-proto/src/types/index_uid.rs b/quickwit/quickwit-proto/src/types/index_uid.rs index 2d964c739bf..2a5d5f098d3 100644 --- a/quickwit/quickwit-proto/src/types/index_uid.rs +++ b/quickwit/quickwit-proto/src/types/index_uid.rs @@ -26,7 +26,7 @@ use serde::{Deserialize, Deserializer, Serialize, Serializer}; use thiserror::Error; pub use ulid::Ulid; -use crate::types::pipeline_uid::ULID_SIZE; +use super::ULID_SIZE; use crate::types::IndexId; /// Index identifiers that uniquely identify not only the index, but also diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index f45f3efbc55..631d88e01df 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -29,17 +29,22 @@ use tracing::warn; pub use ulid::Ulid; mod doc_mapping_uid; +mod doc_uid; mod index_uid; mod pipeline_uid; mod position; mod shard_id; pub use doc_mapping_uid::DocMappingUid; +pub use doc_uid::{DocUid, DocUidGenerator}; pub use index_uid::IndexUid; pub use pipeline_uid::PipelineUid; pub use position::Position; pub use shard_id::ShardId; +/// The size of an ULID in bytes. Use `ULID_LEN` for the length of Base32 encoded ULID strings. +pub(crate) const ULID_SIZE: usize = 16; + pub type IndexId = String; pub type SourceId = String; diff --git a/quickwit/quickwit-proto/src/types/pipeline_uid.rs b/quickwit/quickwit-proto/src/types/pipeline_uid.rs index bb2b37669d1..b601d3a95ea 100644 --- a/quickwit/quickwit-proto/src/types/pipeline_uid.rs +++ b/quickwit/quickwit-proto/src/types/pipeline_uid.rs @@ -17,17 +17,18 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::borrow::Cow; use std::fmt; use std::fmt::{Display, Formatter}; use std::str::FromStr; +use serde::de::Error; use serde::{Deserialize, Serialize}; use ulid::Ulid; -/// The size of a ULID in bytes. -pub(crate) const ULID_SIZE: usize = 16; +use super::ULID_SIZE; -/// A pipeline uid identify an indexing pipeline and an indexing task. +/// A pipeline UID identifies an indexing pipeline and an indexing task. #[derive(Clone, Copy, Default, Hash, Eq, PartialEq, Ord, PartialOrd)] pub struct PipelineUid(Ulid); @@ -44,7 +45,7 @@ impl Display for PipelineUid { } impl PipelineUid { - /// Creates a new random pipeline uid. + /// Creates a new random pipeline UID. pub fn random() -> Self { Self(Ulid::new()) } @@ -73,9 +74,8 @@ impl Serialize for PipelineUid { impl<'de> Deserialize<'de> for PipelineUid { fn deserialize>(deserializer: D) -> Result { - let ulid_str = String::deserialize(deserializer)?; - let ulid = Ulid::from_string(&ulid_str) - .map_err(|error| serde::de::Error::custom(error.to_string()))?; + let ulid_str: Cow<'de, str> = Cow::deserialize(deserializer)?; + let ulid = Ulid::from_string(&ulid_str).map_err(D::Error::custom)?; Ok(Self(ulid)) } } diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index d00d882c141..0cdc0c80ac3 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -38,7 +38,7 @@ use crate::with_arg; #[derive(utoipa::OpenApi)] #[openapi( paths(get_delete_tasks, post_delete_request), - components(schemas(DeleteQueryRequest, DeleteTask, DeleteQuery,)) + components(schemas(DeleteQueryRequest, DeleteTask, DeleteQuery)) )] pub struct DeleteTaskApi; diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs index c1aa0c19018..c33bbb1e041 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs @@ -24,14 +24,14 @@ use hyper::StatusCode; use quickwit_config::INGEST_V2_SOURCE_ID; use quickwit_ingest::IngestRequestV2Builder; use quickwit_proto::ingest::router::{ - IngestFailureReason, IngestRouterService, IngestRouterServiceClient, + IngestFailureReason, IngestResponseV2, IngestRouterService, IngestRouterServiceClient, }; use quickwit_proto::ingest::CommitTypeV2; -use quickwit_proto::types::IndexId; +use quickwit_proto::types::{DocUid, IndexId}; use serde::{Deserialize, Serialize}; use tracing::warn; -use super::model::ErrorCauseException; +use super::model::ElasticException; use crate::elasticsearch_api::model::{BulkAction, ElasticBulkOptions, ElasticsearchError}; use crate::ingest_api::lines; use crate::Body; @@ -69,10 +69,21 @@ pub(crate) struct ElasticBulkError { #[serde(rename = "index")] pub index_id: Option, #[serde(rename = "type")] - pub exception: ErrorCauseException, + pub exception: ElasticException, pub reason: String, } +type ElasticDocId = String; + +#[derive(Debug)] +struct DocHandle { + doc_uid: DocUid, + es_doc_id: Option, + // Whether the document failed to parse. When the struct is instantiated, this value is set to + // `false` and then mutated if the ingest response contains a parse failure for this document. + is_parse_failure: bool, +} + pub(crate) async fn elastic_bulk_ingest_v2( default_index_id: Option, body: Body, @@ -82,21 +93,21 @@ pub(crate) async fn elastic_bulk_ingest_v2( let now = Instant::now(); let mut ingest_request_builder = IngestRequestV2Builder::default(); let mut lines = lines(&body.content).enumerate(); - let mut per_subrequest_id_es_doc_ids: HashMap>> = HashMap::new(); + let mut per_subrequest_doc_handles: HashMap> = HashMap::new(); while let Some((line_no, line)) = lines.next() { let action = serde_json::from_slice::(line).map_err(|error| { ElasticsearchError::new( StatusCode::BAD_REQUEST, format!("Malformed action/metadata line [{}]: {error}", line_no + 1), - Some(ErrorCauseException::IllegalArgument), + Some(ElasticException::IllegalArgument), ) })?; - let (_, source) = lines.next().ok_or_else(|| { + let (_, doc) = lines.next().ok_or_else(|| { ElasticsearchError::new( StatusCode::BAD_REQUEST, "Validation Failed: 1: no requests added;".to_string(), - Some(ErrorCauseException::ActionRequestValidation), + Some(ElasticException::ActionRequestValidation), ) })?; let meta = action.into_meta(); @@ -111,15 +122,20 @@ pub(crate) async fn elastic_bulk_ingest_v2( ElasticsearchError::new( StatusCode::BAD_REQUEST, "Validation Failed: 1: index is missing;".to_string(), - Some(ErrorCauseException::ActionRequestValidation), + Some(ElasticException::ActionRequestValidation), ) })?; - let subrequest_id = ingest_request_builder.add_doc(index_id, source); - - per_subrequest_id_es_doc_ids + let (subrequest_id, doc_uid) = ingest_request_builder.add_doc(index_id, doc); + + let doc_handle = DocHandle { + doc_uid, + es_doc_id: meta.es_doc_id, + is_parse_failure: false, + }; + per_subrequest_doc_handles .entry(subrequest_id) .or_default() - .push(meta.es_doc_id); + .push(doc_handle); } let commit_type: CommitTypeV2 = bulk_options.refresh.into(); @@ -131,27 +147,74 @@ pub(crate) async fn elastic_bulk_ingest_v2( let Some(ingest_request) = ingest_request_opt else { return Ok(ElasticBulkResponse::default()); }; - let ingest_response_v2 = ingest_router.ingest(ingest_request).await?; - let errors = !ingest_response_v2.failures.is_empty(); + let ingest_response = ingest_router.ingest(ingest_request).await?; + make_elastic_bulk_response_v2(ingest_response, per_subrequest_doc_handles, now) +} + +fn make_elastic_bulk_response_v2( + ingest_response_v2: IngestResponseV2, + mut per_subrequest_doc_handles: HashMap>, + now: Instant, +) -> Result { let mut actions: Vec = Vec::new(); + let mut errors = false; + // Populate the items for each `IngestSuccess` subresponse. They may be partially successful and + // contain some parse failures. for success in ingest_response_v2.successes { - let es_doc_ids = per_subrequest_id_es_doc_ids - .remove(&success.subrequest_id) - .ok_or_else(|| { - ElasticsearchError::new( - StatusCode::INTERNAL_SERVER_ERROR, - format!( - "could not find subrequest `{}` in bulk request", - success.subrequest_id - ), - None, - ) - })?; - for es_doc_id in es_doc_ids { + let index_id = success + .index_uid + .map(|index_uid| index_uid.index_id) + .expect("`index_uid` should be a required field"); + + // Find the doc handles for the subresponse. + let mut doc_handles = + remove_doc_handles(&mut per_subrequest_doc_handles, success.subrequest_id)?; + doc_handles.sort_unstable_by(|left, right| left.doc_uid.cmp(&right.doc_uid)); + + // Populate the response items with one error per parse failure. + for parse_failure in success.parse_failures { + errors = true; + + // Since the generated doc UIDs are monotonically increasing, and inserted in order, we + // can find doc handles using binary search. + let doc_handle_idx = doc_handles + .binary_search_by_key(&parse_failure.doc_uid(), |doc_handle| doc_handle.doc_uid) + .map_err(|_| { + ElasticsearchError::new( + StatusCode::INTERNAL_SERVER_ERROR, + format!( + "could not find doc `{}` in bulk request", + parse_failure.doc_uid() + ), + None, + ) + })?; + let doc_handle = &mut doc_handles[doc_handle_idx]; + doc_handle.is_parse_failure = true; + + let error = ElasticBulkError { + index_id: Some(index_id.clone()), + exception: ElasticException::DocumentParsing, + reason: parse_failure.message, + }; let item = ElasticBulkItem { - index_id: success.index_uid().index_id.clone(), - es_doc_id, + index_id: index_id.clone(), + es_doc_id: doc_handle.es_doc_id.take(), + status: StatusCode::BAD_REQUEST, + error: Some(error), + }; + let action = ElasticBulkAction::Index(item); + actions.push(action); + } + // Populate the remaining successful items. + for mut doc_handle in doc_handles { + if doc_handle.is_parse_failure { + continue; + } + let item = ElasticBulkItem { + index_id: index_id.clone(), + es_doc_id: doc_handle.es_doc_id.take(), status: StatusCode::CREATED, error: None, }; @@ -159,59 +222,65 @@ pub(crate) async fn elastic_bulk_ingest_v2( actions.push(action); } } + // Repeat the operation for each `IngestFailure` subresponse. for failure in ingest_response_v2.failures { - let es_doc_ids = per_subrequest_id_es_doc_ids - .remove(&failure.subrequest_id) - .ok_or_else(|| { - ElasticsearchError::new( + errors = true; + + // Find the doc handles for the subrequest. + let doc_handles = + remove_doc_handles(&mut per_subrequest_doc_handles, failure.subrequest_id)?; + + // Populate the response items with one error per doc handle. + let (exception, reason, status) = match failure.reason() { + IngestFailureReason::IndexNotFound => ( + ElasticException::IndexNotFound, + format!("no such index [{}]", failure.index_id), + StatusCode::NOT_FOUND, + ), + IngestFailureReason::SourceNotFound => ( + ElasticException::SourceNotFound, + format!("no such source [{}]", failure.index_id), + StatusCode::NOT_FOUND, + ), + IngestFailureReason::Timeout => ( + ElasticException::Timeout, + format!("timeout [{}]", failure.index_id), + StatusCode::REQUEST_TIMEOUT, + ), + reason => { + let pretty_reason = reason + .as_str_name() + .strip_prefix("INGEST_FAILURE_REASON_") + .unwrap_or("") + .replace('_', " ") + .to_ascii_lowercase(); + ( + ElasticException::Internal, + format!("{} error [{}]", pretty_reason, failure.index_id), StatusCode::INTERNAL_SERVER_ERROR, - format!( - "could not find subrequest `{}` in bulk request", - failure.subrequest_id - ), - None, ) - })?; - match failure.reason() { - IngestFailureReason::IndexNotFound => { - for es_doc_id in es_doc_ids { - let error = ElasticBulkError { - index_id: Some(failure.index_id.clone()), - exception: ErrorCauseException::IndexNotFound, - reason: format!("no such index [{}]", failure.index_id), - }; - let item = ElasticBulkItem { - index_id: failure.index_id.clone(), - es_doc_id, - status: StatusCode::NOT_FOUND, - error: Some(error), - }; - let action = ElasticBulkAction::Index(item); - actions.push(action); - } - } - IngestFailureReason::Timeout => { - for es_doc_id in es_doc_ids { - let error = ElasticBulkError { - index_id: Some(failure.index_id.clone()), - exception: ErrorCauseException::Timeout, - reason: format!("timeout [{}]", failure.index_id), - }; - let item = ElasticBulkItem { - index_id: failure.index_id.clone(), - es_doc_id, - status: StatusCode::REQUEST_TIMEOUT, - error: Some(error), - }; - let action = ElasticBulkAction::Index(item); - actions.push(action); - } - } - _ => { - // TODO } + }; + for mut doc_handle in doc_handles { + let error = ElasticBulkError { + index_id: Some(failure.index_id.clone()), + exception, + reason: reason.clone(), + }; + let item = ElasticBulkItem { + index_id: failure.index_id.clone(), + es_doc_id: doc_handle.es_doc_id.take(), + status, + error: Some(error), + }; + let action = ElasticBulkAction::Index(item); + actions.push(action); } } + assert!( + per_subrequest_doc_handles.is_empty(), + "doc handles should be empty" + ); let took_millis = now.elapsed().as_millis() as u64; let bulk_response = ElasticBulkResponse { @@ -222,12 +291,28 @@ pub(crate) async fn elastic_bulk_ingest_v2( Ok(bulk_response) } +fn remove_doc_handles( + per_subrequest_doc_handles: &mut HashMap>, + subrequest_id: u32, +) -> Result, ElasticsearchError> { + per_subrequest_doc_handles + .remove(&subrequest_id) + .ok_or_else(|| { + ElasticsearchError::new( + StatusCode::INTERNAL_SERVER_ERROR, + format!("could not find subrequest `{subrequest_id}` in bulk request"), + None, + ) + }) +} + #[cfg(test)] mod tests { use quickwit_proto::ingest::router::{ IngestFailure, IngestFailureReason, IngestResponseV2, IngestSuccess, MockIngestRouterService, }; + use quickwit_proto::ingest::{ParseFailure, ParseFailureReason}; use quickwit_proto::types::{IndexUid, Position, ShardId}; use warp::{Filter, Rejection, Reply}; @@ -239,6 +324,36 @@ mod tests { use crate::format::extract_format_from_qs; use crate::with_arg; + impl ElasticBulkAction { + fn index_id(&self) -> &IndexId { + match self { + ElasticBulkAction::Create(item) => &item.index_id, + ElasticBulkAction::Index(item) => &item.index_id, + } + } + + fn es_doc_id(&self) -> Option<&str> { + match self { + ElasticBulkAction::Create(item) => item.es_doc_id.as_deref(), + ElasticBulkAction::Index(item) => item.es_doc_id.as_deref(), + } + } + + fn status(&self) -> StatusCode { + match self { + ElasticBulkAction::Create(item) => item.status, + ElasticBulkAction::Index(item) => item.status, + } + } + + fn error(&self) -> Option<&ElasticBulkError> { + match self { + ElasticBulkAction::Create(item) => item.error.as_ref(), + ElasticBulkAction::Index(item) => item.error.as_ref(), + } + } + } + fn es_compat_bulk_handler_v2( ingest_router: IngestRouterServiceClient, ) -> impl Filter + Clone { @@ -284,6 +399,8 @@ mod tests { source_id: INGEST_V2_SOURCE_ID.to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(1u64)), + num_ingested_docs: 2, + parse_failures: Vec::new(), }, IngestSuccess { subrequest_id: 1, @@ -291,6 +408,8 @@ mod tests { source_id: INGEST_V2_SOURCE_ID.to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), + num_ingested_docs: 1, + parse_failures: Vec::new(), }, ], failures: Vec::new(), @@ -387,6 +506,8 @@ mod tests { source_id: INGEST_V2_SOURCE_ID.to_string(), shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), + num_ingested_docs: 1, + parse_failures: Vec::new(), }], failures: Vec::new(), }) @@ -538,4 +659,100 @@ mod tests { assert!(bulk_response.errors); assert_eq!(bulk_response.actions.len(), 3); } + + #[test] + fn test_make_elastic_bulk_response_v2() { + let response = make_elastic_bulk_response_v2( + IngestResponseV2::default(), + HashMap::new(), + Instant::now(), + ) + .unwrap(); + + assert!(!response.errors); + assert!(response.actions.is_empty()); + + let ingest_response_v2 = IngestResponseV2 { + successes: vec![IngestSuccess { + subrequest_id: 0, + index_uid: Some(IndexUid::for_test("test-index-foo", 0)), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(0)), + replication_position_inclusive: Some(Position::offset(0u64)), + num_ingested_docs: 1, + parse_failures: vec![ParseFailure { + doc_uid: Some(DocUid::for_test(1)), + reason: ParseFailureReason::InvalidJson as i32, + message: "failed to parse JSON document".to_string(), + }], + }], + failures: vec![IngestFailure { + subrequest_id: 1, + index_id: "test-index-bar".to_string(), + source_id: "test-source".to_string(), + reason: IngestFailureReason::IndexNotFound as i32, + }], + }; + let per_request_doc_handles = HashMap::from_iter([ + ( + 0, + vec![ + DocHandle { + doc_uid: DocUid::for_test(0), + es_doc_id: Some("0".to_string()), + is_parse_failure: false, + }, + DocHandle { + doc_uid: DocUid::for_test(1), + es_doc_id: Some("1".to_string()), + is_parse_failure: false, + }, + ], + ), + ( + 1, + vec![DocHandle { + doc_uid: DocUid::for_test(2), + es_doc_id: Some("2".to_string()), + is_parse_failure: false, + }], + ), + ]); + let mut response = make_elastic_bulk_response_v2( + ingest_response_v2, + per_request_doc_handles, + Instant::now(), + ) + .unwrap(); + + assert!(response.errors); + assert_eq!(response.actions.len(), 3); + + response + .actions + .sort_unstable_by(|left, right| left.es_doc_id().cmp(&right.es_doc_id())); + + assert_eq!(response.actions[0].index_id(), "test-index-foo"); + assert_eq!(response.actions[0].es_doc_id(), Some("0")); + assert_eq!(response.actions[0].status(), StatusCode::CREATED); + assert!(response.actions[0].error().is_none()); + + assert_eq!(response.actions[1].index_id(), "test-index-foo"); + assert_eq!(response.actions[1].es_doc_id(), Some("1")); + assert_eq!(response.actions[1].status(), StatusCode::BAD_REQUEST); + + let error = response.actions[1].error().unwrap(); + assert_eq!(error.index_id.as_ref().unwrap(), "test-index-foo"); + assert_eq!(error.exception, ElasticException::DocumentParsing); + assert_eq!(error.reason, "failed to parse JSON document"); + + assert_eq!(response.actions[2].index_id(), "test-index-bar"); + assert_eq!(response.actions[2].es_doc_id(), Some("2")); + assert_eq!(response.actions[2].status(), StatusCode::NOT_FOUND); + + let error = response.actions[2].error().unwrap(); + assert_eq!(error.index_id.as_ref().unwrap(), "test-index-bar"); + assert_eq!(error.exception, ElasticException::IndexNotFound); + assert_eq!(error.reason, "no such index [test-index-bar]"); + } } diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/model/error.rs b/quickwit/quickwit-serve/src/elasticsearch_api/model/error.rs index 53704afc3f6..2c1964968e5 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/model/error.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/model/error.rs @@ -38,12 +38,12 @@ impl ElasticsearchError { pub fn new( status: StatusCode, reason: String, - exception_opt: Option, + exception_opt: Option, ) -> Self { if status.is_server_error() { - rate_limited_error!(limit_per_min=10, status=%status, "http request failed with internal server error: {}", reason); + rate_limited_error!(limit_per_min=10, status=%status, "http request failed with server error: {reason}"); } else if !status.is_success() { - rate_limited_debug!(limit_per_min=10, status=%status, "http request failed: {}", reason); + rate_limited_debug!(limit_per_min=10, status=%status, "http request failed: {reason}"); } ElasticsearchError { status, @@ -140,24 +140,35 @@ impl From for ElasticsearchError { } } -#[derive(Debug, Clone, Copy, Serialize, Deserialize)] -pub enum ErrorCauseException { +#[derive(Debug, Clone, Copy, Eq, PartialEq, Serialize, Deserialize)] +pub enum ElasticException { #[serde(rename = "action_request_validation_exception")] ActionRequestValidation, + #[serde(rename = "document_parsing_exception")] + DocumentParsing, + // This is an exception proper to Quickwit. + #[serde(rename = "internal_exception")] + Internal, #[serde(rename = "illegal_argument_exception")] IllegalArgument, #[serde(rename = "index_not_found_exception")] IndexNotFound, + // This is an exception proper to Quickwit. + #[serde(rename = "source_not_found_exception")] + SourceNotFound, #[serde(rename = "timeout_exception")] Timeout, } -impl ErrorCauseException { +impl ElasticException { pub fn as_str(&self) -> &'static str { match self { Self::ActionRequestValidation => "action_request_validation_exception", + Self::DocumentParsing => "document_parsing_exception", + Self::Internal => "internal_exception", Self::IllegalArgument => "illegal_argument_exception", Self::IndexNotFound => "index_not_found_exception", + Self::SourceNotFound => "source_not_found_exception", Self::Timeout => "timeout_exception", } } diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/model/mod.rs b/quickwit/quickwit-serve/src/elasticsearch_api/model/mod.rs index 49dda496892..ec7957f942f 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/model/mod.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/model/mod.rs @@ -34,7 +34,7 @@ pub use cat_indices::{ CatIndexQueryParams, ElasticsearchCatIndexResponse, ElasticsearchResolveIndexEntryResponse, ElasticsearchResolveIndexResponse, }; -pub use error::{ElasticsearchError, ErrorCauseException}; +pub use error::{ElasticException, ElasticsearchError}; pub use field_capability::{ build_list_field_request_for_es_api, convert_to_es_field_capabilities_response, FieldCapabilityQueryParams, FieldCapabilityRequestBody, FieldCapabilityResponse, diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index afd0bfb752b..5522f94ea04 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -27,7 +27,7 @@ use quickwit_proto::ingest::router::{ IngestFailureReason, IngestRequestV2, IngestResponseV2, IngestRouterService, IngestRouterServiceClient, IngestSubrequest, }; -use quickwit_proto::types::IndexId; +use quickwit_proto::types::{DocUidGenerator, IndexId}; use serde::Deserialize; use warp::{Filter, Rejection}; @@ -122,9 +122,10 @@ async fn ingest_v2( ingest_router: IngestRouterServiceClient, ) -> Result { let mut doc_batch_builder = DocBatchV2Builder::default(); + let mut doc_uid_generator = DocUidGenerator::default(); for doc in lines(&body.content) { - doc_batch_builder.add_doc(doc); + doc_batch_builder.add_doc(doc_uid_generator.next_doc_uid(), doc); } let doc_batch_opt = doc_batch_builder.build(); diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0001-happy-path.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0001-happy-path.yaml index 589288f8c56..42fd7c0b18e 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0001-happy-path.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0001-happy-path.yaml @@ -1,16 +1,16 @@ ndjson: - - index: { "_index": "test-index-happy-path", "_id": "1" } + - index: { "_index": "test-index", "_id": "1" } - message: Hello, World! - - index: { "_index": "test-index-happy-path" } + - index: { "_index": "test-index" } - message: Hola, Mundo! status_code: 200 expected: errors: false items: - index: - _index: test-index-happy-path + _index: test-index _id: "1" status: 201 - index: - _index: test-index-happy-path + _index: test-index status: 201 diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-put-request.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-put-request.yaml index a5a8a83268d..6299360c69b 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-put-request.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-put-request.yaml @@ -1,4 +1,11 @@ method: PUT ndjson: - - index: { "_index": "test-index", "_id": "1" } + - index: { "_index": "test-index" } - message: Hello, World! +status_code: 200 +expected: + errors: false + items: + - index: + _index: test-index + status: 201 diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-validation-failed-no-requests-added.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-validation-failed-no-requests-added.yaml deleted file mode 100644 index c233ff577af..00000000000 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0004-validation-failed-no-requests-added.yaml +++ /dev/null @@ -1,8 +0,0 @@ -ndjson: - - index: { "_index": "test-index", "_id": "2" } -status_code: 400 -expected: - status: 400 - error: - type: action_request_validation_exception - reason: "Validation Failed: 1: no requests added;" diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0005-document-parsing-exception.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0005-document-parsing-exception.yaml new file mode 100644 index 00000000000..0a005c7c2ef --- /dev/null +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/0005-document-parsing-exception.yaml @@ -0,0 +1,16 @@ +ndjson: + - index: { "_index": "test-index", "_id": "5" } + - message: Hello, World! + timestamp: timestamp +status_code: 200 +expected: + errors: true + items: + - index: + _index: test-index + _id: "5" + status: 400 + error: + type: document_parsing_exception + reason: + $expect: "'timestamp' in val" diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.elasticsearch.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.elasticsearch.yaml index 15bdb58fee6..1cc65fa967d 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.elasticsearch.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.elasticsearch.yaml @@ -11,6 +11,10 @@ json: { "message": { "type": "text", "store": true + }, + "timestamp": { + "type": "integer", + "store": true } } } diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.quickwit.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.quickwit.yaml index 3d3997dc417..abeb9a76019 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.quickwit.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_setup.quickwit.yaml @@ -14,6 +14,8 @@ json: field_mappings: - name: message type: text + - name: timestamp + type: datetime sleep_after: 3 --- # Create index template diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.elasticsearch.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.elasticsearch.yaml index 798b8da14d5..e69de29bb2d 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.elasticsearch.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.elasticsearch.yaml @@ -1,3 +0,0 @@ -method: DELETE -endpoint: test-index ----