From d684b92fe516d21d787c73afa3c2c60b36b6aabd Mon Sep 17 00:00:00 2001 From: trinity-1686a Date: Fri, 30 Aug 2024 17:55:12 +0200 Subject: [PATCH 01/25] allow querying non existing fields (#5308) * make integration test idempotent * ignore missing field on lenient query --- .../quickwit-doc-mapper/src/doc_mapper.rs | 1 + .../quickwit-doc-mapper/src/tag_pruning.rs | 1 + quickwit/quickwit-jaeger/src/lib.rs | 3 +- .../file-backed-index/v0.9.expected.json | 2 +- .../test-data/file-backed-index/v0.9.json | 2 +- .../elastic_query_dsl/match_bool_prefix.rs | 1 + .../elastic_query_dsl/match_phrase_query.rs | 2 ++ .../src/elastic_query_dsl/match_query.rs | 16 ++++++----- .../src/elastic_query_dsl/multi_match.rs | 28 +++++++++---------- .../elastic_query_dsl/query_string_query.rs | 20 +++++++------ .../src/query_ast/full_text_query.rs | 6 ++++ quickwit/quickwit-query/src/query_ast/mod.rs | 6 ++++ .../src/query_ast/term_query.rs | 1 + .../src/query_ast/user_input_query.rs | 28 +++++++++++++++++-- .../quickwit-query/src/query_ast/utils.rs | 10 ++++++- quickwit/quickwit-search/src/root.rs | 1 + .../src/delete_task_api/handler.rs | 2 +- .../src/elasticsearch_api/rest_handler.rs | 2 ++ .../0005-query_string_query.yaml | 18 ++++++++++++ .../bulk/_setup.quickwit.yaml | 7 ++++- .../bulk/_teardown.quickwit.yaml | 4 +++ 21 files changed, 123 insertions(+), 38 deletions(-) diff --git a/quickwit/quickwit-doc-mapper/src/doc_mapper.rs b/quickwit/quickwit-doc-mapper/src/doc_mapper.rs index 307a8e7d449..b58d3edaa7c 100644 --- a/quickwit/quickwit-doc-mapper/src/doc_mapper.rs +++ b/quickwit/quickwit-doc-mapper/src/doc_mapper.rs @@ -361,6 +361,7 @@ mod tests { user_text: "json_field.toto.titi:hello".to_string(), default_fields: None, default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&[]) .unwrap(); diff --git a/quickwit/quickwit-doc-mapper/src/tag_pruning.rs b/quickwit/quickwit-doc-mapper/src/tag_pruning.rs index 08ece78f029..e3b2255efc3 100644 --- a/quickwit/quickwit-doc-mapper/src/tag_pruning.rs +++ b/quickwit/quickwit-doc-mapper/src/tag_pruning.rs @@ -398,6 +398,7 @@ mod test { user_text: user_query.to_string(), default_fields: None, default_operator: BooleanOperand::Or, + lenient: false, } .into(); let parsed_query_ast = query_ast.parse_user_query(&[]).unwrap(); diff --git a/quickwit/quickwit-jaeger/src/lib.rs b/quickwit/quickwit-jaeger/src/lib.rs index 7e5970de4c9..220b6f6512c 100644 --- a/quickwit/quickwit-jaeger/src/lib.rs +++ b/quickwit/quickwit-jaeger/src/lib.rs @@ -1182,7 +1182,8 @@ mod tests { quickwit_query::query_ast::UserInputQuery { user_text: "query".to_string(), default_fields: None, - default_operator: quickwit_query::BooleanOperand::And + default_operator: quickwit_query::BooleanOperand::And, + lenient: false, } .into() ); diff --git a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.expected.json b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.expected.json index a9d233c72dd..ffa5699989d 100644 --- a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.expected.json +++ b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.expected.json @@ -190,7 +190,7 @@ "opstamp": 10, "delete_query": { "index_uid": "my-index:00000000000000000000000001", - "query_ast": "{\"type\":\"bool\",\"must\":[{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Harry\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}}},{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Potter\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}}}]}" + "query_ast": "{\"type\":\"bool\",\"must\":[{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Harry\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}},\"lenient\":false},{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Potter\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}},\"lenient\":false}]}" } } ] diff --git a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.json b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.json index a9d233c72dd..ffa5699989d 100644 --- a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.json +++ b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.9.json @@ -190,7 +190,7 @@ "opstamp": 10, "delete_query": { "index_uid": "my-index:00000000000000000000000001", - "query_ast": "{\"type\":\"bool\",\"must\":[{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Harry\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}}},{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Potter\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}}}]}" + "query_ast": "{\"type\":\"bool\",\"must\":[{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Harry\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}},\"lenient\":false},{\"type\":\"full_text\",\"field\":\"body\",\"text\":\"Potter\",\"params\":{\"mode\":{\"type\":\"phrase_fallback_to_intersection\"}},\"lenient\":false}]}" } } ] diff --git a/quickwit/quickwit-query/src/elastic_query_dsl/match_bool_prefix.rs b/quickwit/quickwit-query/src/elastic_query_dsl/match_bool_prefix.rs index 9b246ff0e08..8c6f56fb948 100644 --- a/quickwit/quickwit-query/src/elastic_query_dsl/match_bool_prefix.rs +++ b/quickwit/quickwit-query/src/elastic_query_dsl/match_bool_prefix.rs @@ -48,6 +48,7 @@ impl ConvertibleToQueryAst for MatchBoolPrefixQuery { field: self.field, text: self.params.query, params: full_text_params, + lenient: self.params.lenient, })) } } diff --git a/quickwit/quickwit-query/src/elastic_query_dsl/match_phrase_query.rs b/quickwit/quickwit-query/src/elastic_query_dsl/match_phrase_query.rs index fdd99a97065..e4b746aa90a 100644 --- a/quickwit/quickwit-query/src/elastic_query_dsl/match_phrase_query.rs +++ b/quickwit/quickwit-query/src/elastic_query_dsl/match_phrase_query.rs @@ -59,6 +59,7 @@ impl ConvertibleToQueryAst for MatchPhraseQuery { field: self.field, text: self.params.query, params: full_text_params, + lenient: false, })) } } @@ -159,6 +160,7 @@ mod tests { field, text, params, + lenient: _, }) = ast else { panic!() diff --git a/quickwit/quickwit-query/src/elastic_query_dsl/match_query.rs b/quickwit/quickwit-query/src/elastic_query_dsl/match_query.rs index 6db5ac0b618..18c565976f7 100644 --- a/quickwit/quickwit-query/src/elastic_query_dsl/match_query.rs +++ b/quickwit/quickwit-query/src/elastic_query_dsl/match_query.rs @@ -42,11 +42,11 @@ pub(crate) struct MatchQueryParams { pub(crate) operator: BooleanOperand, #[serde(default)] pub(crate) zero_terms_query: MatchAllOrNone, - // Regardless of this option Quickwit behaves in elasticsearch definition of - // lenient. We include this property here just to accept user queries containing - // this option. - #[serde(default, rename = "lenient")] - pub(crate) _lenient: bool, + // Quickwit and Elastic have different notions of lenient. For us, it means it's okay to + // disregard part of the query where which uses non-existing collumn (which Elastic does by + // default). For Elastic, it covers type errors (searching text in an integer field). + #[serde(default)] + pub(crate) lenient: bool, } impl ConvertibleToQueryAst for MatchQuery { @@ -60,6 +60,7 @@ impl ConvertibleToQueryAst for MatchQuery { field: self.field, text: self.params.query, params: full_text_params, + lenient: self.params.lenient, })) } } @@ -88,7 +89,7 @@ impl From for MatchQueryParams { query, zero_terms_query: Default::default(), operator: Default::default(), - _lenient: false, + lenient: false, } } } @@ -137,7 +138,7 @@ mod tests { query: "hello".to_string(), operator: BooleanOperand::And, zero_terms_query: crate::MatchAllOrNone::MatchAll, - _lenient: false, + lenient: false, }, }; let ast = match_query.convert_to_query_ast().unwrap(); @@ -145,6 +146,7 @@ mod tests { field, text, params, + lenient: _, }) = ast else { panic!() diff --git a/quickwit/quickwit-query/src/elastic_query_dsl/multi_match.rs b/quickwit/quickwit-query/src/elastic_query_dsl/multi_match.rs index 14aed6f1442..9b607151a31 100644 --- a/quickwit/quickwit-query/src/elastic_query_dsl/multi_match.rs +++ b/quickwit/quickwit-query/src/elastic_query_dsl/multi_match.rs @@ -30,7 +30,7 @@ use crate::elastic_query_dsl::phrase_prefix_query::{ }; use crate::elastic_query_dsl::{ConvertibleToQueryAst, ElasticQueryDslInner}; -/// Multi match queries are a bit odd. They end up being expanded into another type query of query. +/// Multi match queries are a bit odd. They end up being expanded into another type of query. /// In Quickwit, we operate this expansion in generic way at the time of deserialization. #[derive(Deserialize, Debug, Eq, PartialEq, Clone)] #[serde(try_from = "MultiMatchQueryForDeserialization")] @@ -48,11 +48,11 @@ struct MultiMatchQueryForDeserialization { #[serde_as(deserialize_as = "OneOrMany<_, PreferMany>")] #[serde(default)] fields: Vec, - // Regardless of this option Quickwit behaves in elasticsearch definition of - // lenient. We include this property here just to accept user queries containing - // this option. - #[serde(default, rename = "lenient")] - _lenient: bool, + // Quickwit and Elastic have different notions of lenient. For us, it means it's okay to + // disregard part of the query where which uses non-existing collumn (which Elastic does by + // default). For Elastic, it covers type errors (searching text in an integer field). + #[serde(default)] + lenient: bool, } fn deserialize_match_query_for_one_field( @@ -198,7 +198,7 @@ mod tests { query: "quick brown fox".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -208,7 +208,7 @@ mod tests { query: "quick brown fox".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -228,7 +228,7 @@ mod tests { query: "quick brown fox".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -238,7 +238,7 @@ mod tests { query: "quick brown fox".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -258,7 +258,7 @@ mod tests { query: "quick brown fox".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -268,7 +268,7 @@ mod tests { query: "quick brown fox".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -350,7 +350,7 @@ mod tests { query: "quick brown".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), @@ -360,7 +360,7 @@ mod tests { query: "quick brown".to_string(), operator: crate::BooleanOperand::Or, zero_terms_query: Default::default(), - _lenient: false, + lenient: false, }, } .into(), diff --git a/quickwit/quickwit-query/src/elastic_query_dsl/query_string_query.rs b/quickwit/quickwit-query/src/elastic_query_dsl/query_string_query.rs index 1df29b78a07..f7192f8928e 100644 --- a/quickwit/quickwit-query/src/elastic_query_dsl/query_string_query.rs +++ b/quickwit/quickwit-query/src/elastic_query_dsl/query_string_query.rs @@ -43,8 +43,8 @@ pub(crate) struct QueryStringQuery { // Regardless of this option Quickwit behaves in elasticsearch definition of // lenient. We include this property here just to accept user queries containing // this option. - #[serde(default, rename = "lenient")] - _lenient: bool, + #[serde(default)] + lenient: bool, } impl ConvertibleToQueryAst for QueryStringQuery { @@ -60,6 +60,7 @@ impl ConvertibleToQueryAst for QueryStringQuery { user_text: self.query, default_fields, default_operator: self.default_operator, + lenient: self.lenient, }; Ok(user_text_query.into()) } @@ -79,7 +80,7 @@ mod tests { default_operator: crate::BooleanOperand::Or, default_field: None, boost: None, - _lenient: false, + lenient: false, }; let QueryAst::UserInput(user_input_query) = query_string_query.convert_to_query_ast().unwrap() @@ -101,7 +102,7 @@ mod tests { default_operator: crate::BooleanOperand::Or, default_field: Some("hello".to_string()), boost: None, - _lenient: false, + lenient: false, }; let QueryAst::UserInput(user_input_query) = query_string_query.convert_to_query_ast().unwrap() @@ -123,7 +124,7 @@ mod tests { default_operator: crate::BooleanOperand::Or, default_field: Some("hello".to_string()), boost: None, - _lenient: false, + lenient: false, }; let err_msg = query_string_query .convert_to_query_ast() @@ -140,7 +141,7 @@ mod tests { default_field: None, default_operator: crate::BooleanOperand::And, boost: None, - _lenient: false, + lenient: false, }; let QueryAst::UserInput(user_input_query) = query_string_query.convert_to_query_ast().unwrap() @@ -158,7 +159,7 @@ mod tests { default_field: None, default_operator: crate::BooleanOperand::Or, boost: None, - _lenient: false, + lenient: false, }; let QueryAst::UserInput(user_input_query) = query_string_query.convert_to_query_ast().unwrap() @@ -177,7 +178,7 @@ mod tests { default_field: None, default_operator: crate::BooleanOperand::Or, boost: None, - _lenient: false, + lenient: false, }; let QueryAst::UserInput(user_input_query) = query_string_query.convert_to_query_ast().unwrap() @@ -200,7 +201,8 @@ mod tests { assert!(matches!(query_ast, QueryAst::UserInput(UserInputQuery { user_text, default_fields, - default_operator + default_operator, + lenient: _, }) if user_text == "hello world" && default_operator == BooleanOperand::Or && default_fields == Some(vec!["text".to_string()]))); diff --git a/quickwit/quickwit-query/src/query_ast/full_text_query.rs b/quickwit/quickwit-query/src/query_ast/full_text_query.rs index 2e809cdd476..d77b39e67df 100644 --- a/quickwit/quickwit-query/src/query_ast/full_text_query.rs +++ b/quickwit/quickwit-query/src/query_ast/full_text_query.rs @@ -227,6 +227,7 @@ pub struct FullTextQuery { pub field: String, pub text: String, pub params: FullTextParams, + pub lenient: bool, } impl From for QueryAst { @@ -249,6 +250,7 @@ impl BuildTantivyAst for FullTextQuery { &self.params, schema, tokenizer_manager, + self.lenient, ) } } @@ -323,6 +325,7 @@ mod tests { mode: BooleanOperand::And.into(), zero_terms_query: crate::MatchAllOrNone::MatchAll, }, + lenient: false, }; let mut schema_builder = Schema::builder(); schema_builder.add_text_field("body", TEXT); @@ -348,6 +351,7 @@ mod tests { mode: FullTextMode::Phrase { slop: 1 }, zero_terms_query: crate::MatchAllOrNone::MatchAll, }, + lenient: false, }; let mut schema_builder = Schema::builder(); schema_builder.add_text_field("body", TEXT); @@ -378,6 +382,7 @@ mod tests { mode: FullTextMode::Phrase { slop: 1 }, zero_terms_query: crate::MatchAllOrNone::MatchAll, }, + lenient: false, }; let mut schema_builder = Schema::builder(); schema_builder.add_text_field("body", TEXT); @@ -407,6 +412,7 @@ mod tests { mode: BooleanOperand::And.into(), zero_terms_query: crate::MatchAllOrNone::MatchAll, }, + lenient: false, }; let mut schema_builder = Schema::builder(); schema_builder.add_text_field("body", TEXT); diff --git a/quickwit/quickwit-query/src/query_ast/mod.rs b/quickwit/quickwit-query/src/query_ast/mod.rs index 9c39c14123f..54213a17d48 100644 --- a/quickwit/quickwit-query/src/query_ast/mod.rs +++ b/quickwit/quickwit-query/src/query_ast/mod.rs @@ -316,6 +316,7 @@ pub fn query_ast_from_user_text(user_text: &str, default_fields: Option>, pub default_operator: BooleanOperand, + pub lenient: bool, } impl UserInputQuery { @@ -73,7 +74,12 @@ impl UserInputQuery { BooleanOperand::And => Occur::Must, BooleanOperand::Or => Occur::Should, }; - convert_user_input_ast_to_query_ast(user_input_ast, default_occur, search_fields) + convert_user_input_ast_to_query_ast( + user_input_ast, + default_occur, + search_fields, + self.lenient, + ) } } @@ -95,10 +101,13 @@ impl BuildTantivyAst for UserInputQuery { } } +/// Convert the AST of a text query to a QueryAst, filling in default field and default occur when +/// they were not present. fn convert_user_input_ast_to_query_ast( user_input_ast: UserInputAst, default_occur: Occur, default_search_fields: &[String], + lenient: bool, ) -> anyhow::Result { match user_input_ast { UserInputAst::Clause(clause) => { @@ -108,6 +117,7 @@ fn convert_user_input_ast_to_query_ast( sub_ast, default_occur, default_search_fields, + lenient, )?; let children_ast_for_occur: &mut Vec = match occur_opt.unwrap_or(default_occur) { @@ -121,7 +131,7 @@ fn convert_user_input_ast_to_query_ast( } UserInputAst::Leaf(leaf) => match *leaf { UserInputLeaf::Literal(literal) => { - convert_user_input_literal(literal, default_search_fields) + convert_user_input_literal(literal, default_search_fields, lenient) } UserInputLeaf::All => Ok(QueryAst::MatchAll), UserInputLeaf::Range { @@ -178,6 +188,7 @@ fn convert_user_input_ast_to_query_ast( *underlying, default_occur, default_search_fields, + lenient, )?; let boost: NotNaNf32 = (boost as f32) .try_into() @@ -215,9 +226,12 @@ fn is_wildcard(phrase: &str) -> bool { .is_break() } +/// Convert a leaf of a text query AST to a QueryAst. +/// This may generate more than a single leaf if there are multiple default fields. fn convert_user_input_literal( user_input_literal: UserInputLiteral, default_search_fields: &[String], + lenient: bool, ) -> anyhow::Result { let UserInputLiteral { field_name, @@ -272,6 +286,7 @@ fn convert_user_input_literal( field: field_name, text: phrase.clone(), params: full_text_params.clone(), + lenient, } .into() } @@ -303,6 +318,7 @@ mod tests { user_text: "hello".to_string(), default_fields: None, default_operator: BooleanOperand::And, + lenient: false, }; let schema = tantivy::schema::Schema::builder().build(); { @@ -336,6 +352,7 @@ mod tests { user_text: "hello".to_string(), default_fields: None, default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&[]) .unwrap_err(); @@ -349,6 +366,7 @@ mod tests { user_text: "hello".to_string(), default_fields: Some(Vec::new()), default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&[]) .unwrap_err(); @@ -365,6 +383,7 @@ mod tests { user_text: "hello".to_string(), default_fields: None, default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&["defaultfield".to_string()]) .unwrap(); @@ -385,6 +404,7 @@ mod tests { user_text: "field:\"hello\"*".to_string(), default_fields: None, default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&[]) .unwrap(); @@ -406,6 +426,7 @@ mod tests { user_text: "hello".to_string(), default_fields: Some(vec!["defaultfield".to_string()]), default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&["defaultfieldweshouldignore".to_string()]) .unwrap(); @@ -426,6 +447,7 @@ mod tests { user_text: "hello".to_string(), default_fields: Some(vec!["fielda".to_string(), "fieldb".to_string()]), default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&["defaultfieldweshouldignore".to_string()]) .unwrap(); @@ -441,6 +463,7 @@ mod tests { user_text: "myfield:hello".to_string(), default_fields: Some(vec!["fieldtoignore".to_string()]), default_operator: BooleanOperand::And, + lenient: false, } .parse_user_query(&["fieldtoignore".to_string()]) .unwrap(); @@ -462,6 +485,7 @@ mod tests { user_text: query.to_string(), default_fields: None, default_operator: BooleanOperand::Or, + lenient: false, } .parse_user_query(&[]) .unwrap(); diff --git a/quickwit/quickwit-query/src/query_ast/utils.rs b/quickwit/quickwit-query/src/query_ast/utils.rs index be2bdf2f3f8..310056c0309 100644 --- a/quickwit/quickwit-query/src/query_ast/utils.rs +++ b/quickwit/quickwit-query/src/query_ast/utils.rs @@ -30,6 +30,7 @@ use crate::query_ast::full_text_query::FullTextParams; use crate::query_ast::tantivy_query_ast::{TantivyBoolQuery, TantivyQueryAst}; use crate::tokenizers::TokenizerManager; use crate::InvalidQuery; +use crate::MatchAllOrNone::MatchNone as TantivyEmptyQuery; const DYNAMIC_FIELD_NAME: &str = "_dynamic"; @@ -71,8 +72,15 @@ pub(crate) fn full_text_query( full_text_params: &FullTextParams, schema: &TantivySchema, tokenizer_manager: &TokenizerManager, + lenient: bool, ) -> Result { - let (field, field_entry, path) = find_field_or_hit_dynamic(full_path, schema)?; + let (field, field_entry, path) = match find_field_or_hit_dynamic(full_path, schema) { + Ok(res) => res, + Err(InvalidQuery::FieldDoesNotExist { .. }) if lenient => { + return Ok(TantivyEmptyQuery.into()) + } + Err(e) => return Err(e), + }; compute_query_with_field( field, field_entry, diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index dfed9722c9c..25bc6baf115 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -3951,6 +3951,7 @@ mod tests { mode: FullTextMode::PhraseFallbackToIntersection, zero_terms_query: MatchAllOrNone::MatchNone, }, + lenient: false, },), tantivy_ast: r#"BooleanQuery { subqueries: [ diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index 0cdc0c80ac3..eae5d625160 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -223,7 +223,7 @@ mod tests { assert_eq!(created_delete_query.index_uid(), &test_sandbox.index_uid()); assert_eq!( created_delete_query.query_ast, - r#"{"type":"full_text","field":"body","text":"myterm","params":{"mode":{"type":"phrase_fallback_to_intersection"}}}"# + r#"{"type":"full_text","field":"body","text":"myterm","params":{"mode":{"type":"phrase_fallback_to_intersection"}},"lenient":false}"# ); assert_eq!(created_delete_query.start_timestamp, Some(1)); assert_eq!(created_delete_query.end_timestamp, Some(10)); diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs index caea3f66e8d..e5caa8703bc 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs @@ -254,6 +254,7 @@ fn build_request_for_es_api( user_text: q.to_string(), default_fields: None, default_operator, + lenient: false, }; user_text_query.into() } else if let Some(query_dsl) = search_body.query { @@ -272,6 +273,7 @@ fn build_request_for_es_api( user_text: query.to_string(), default_fields: None, default_operator, + lenient: false, }; QueryAst::UserInput(user_text_query) }) diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/0005-query_string_query.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/0005-query_string_query.yaml index d341b8c6b97..53079e9f7eb 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/0005-query_string_query.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/0005-query_string_query.yaml @@ -221,3 +221,21 @@ expected: hits: total: value: 1 +--- +json: + query: + query_string: + query: "true" + fields: ["public", "public.inner"] + lenient: true +expected: + hits: + total: + value: 100 +--- +json: + query: + query_string: + query: "true" + fields: ["public", "public.inner"] +status_code: 400 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 abeb9a76019..b1a48066ca5 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 @@ -1,9 +1,14 @@ -# Delete possibly remaining index +# Delete possibly remaining index and template method: DELETE api_root: http://localhost:7280/api/v1/ endpoint: indexes/test-index status_code: null --- +method: DELETE +api_root: http://localhost:7280/api/v1/ +endpoint: templates/test-index-template +status_code: null +--- method: POST api_root: http://localhost:7280/api/v1/ endpoint: indexes/ diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.quickwit.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.quickwit.yaml index ee1b3982ec3..4accabf684c 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.quickwit.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/bulk/_teardown.quickwit.yaml @@ -1,3 +1,7 @@ method: DELETE api_root: http://localhost:7280/api/v1/ endpoint: indexes/test-index +--- +method: DELETE +api_root: http://localhost:7280/api/v1/ +endpoint: templates/test-index-template From ab1a7821dba45e404f0efc41742ec3102ae46d97 Mon Sep 17 00:00:00 2001 From: trinity-1686a Date: Fri, 30 Aug 2024 21:13:03 +0200 Subject: [PATCH 02/25] improve trace search backward compatibility (#5371) --- quickwit/quickwit-jaeger/src/lib.rs | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/quickwit/quickwit-jaeger/src/lib.rs b/quickwit/quickwit-jaeger/src/lib.rs index 220b6f6512c..3efa278c5db 100644 --- a/quickwit/quickwit-jaeger/src/lib.rs +++ b/quickwit/quickwit-jaeger/src/lib.rs @@ -45,7 +45,8 @@ use quickwit_proto::jaeger::storage::v1::{ }; use quickwit_proto::opentelemetry::proto::trace::v1::status::StatusCode as OtlpStatusCode; use quickwit_proto::search::{CountHits, ListTermsRequest, SearchRequest}; -use quickwit_query::query_ast::{BoolQuery, QueryAst, RangeQuery, TermQuery}; +use quickwit_query::query_ast::{BoolQuery, QueryAst, RangeQuery, TermQuery, UserInputQuery}; +use quickwit_query::BooleanOperand; use quickwit_search::{FindTraceIdsCollector, SearchService}; use serde::Deserialize; use serde_json::Value as JsonValue; @@ -321,13 +322,15 @@ impl JaegerService { query.should.push(term_query.into()); } if root_only { - // TODO this isn't backward compatible. We could do NOT is_root:false with a lenient - // UserInputQuery once we support being lenient on missing fields - let term_query = TermQuery { - field: "is_root".to_string(), - value: "true".to_string(), + // we do this so we don't error on old indexes, and instead return both root and non + // root spans + let is_root = UserInputQuery { + user_text: "NOT is_root:false".to_string(), + default_fields: None, + default_operator: BooleanOperand::And, + lenient: true, }; - query.must.push(term_query.into()); + query.must.push(is_root.into()); } let query_ast: QueryAst = query.into(); From 34a162f28f3ab116ac7cbf469dfdc97041e0b3ce Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 2 Sep 2024 14:27:27 +0200 Subject: [PATCH 03/25] Fix flaky integ tests (#5348) * Pass TcpListener to quickwit_serve in integ tests * Use static dispatch instead of enum * Simplify cluster sandbox API * Increase wait_for_split to match higher commit timeouts --- quickwit/quickwit-cli/src/service.rs | 2 + .../quickwit-config/src/node_config/mod.rs | 10 +- .../src/node_config/serialize.rs | 9 +- .../quickwit-integration-tests/Cargo.toml | 2 +- .../src/test_utils/cluster_sandbox.rs | 424 +++++++++--------- .../src/test_utils/mod.rs | 3 +- .../src/test_utils/shutdown.rs | 73 +++ .../src/tests/basic_tests.rs | 35 +- .../src/tests/ingest_tests.rs | 97 ++-- .../src/tests/otlp_tests.rs | 56 ++- .../src/tests/sqs_tests.rs | 6 +- .../tests/update_tests/doc_mapping_tests.rs | 4 +- .../update_tests/search_settings_tests.rs | 22 +- quickwit/quickwit-serve/Cargo.toml | 2 +- quickwit/quickwit-serve/src/grpc.rs | 12 +- quickwit/quickwit-serve/src/lib.rs | 7 +- quickwit/quickwit-serve/src/rest.rs | 10 +- quickwit/quickwit-serve/src/tcp_listener.rs | 81 ++++ 18 files changed, 488 insertions(+), 367 deletions(-) create mode 100644 quickwit/quickwit-integration-tests/src/test_utils/shutdown.rs create mode 100644 quickwit/quickwit-serve/src/tcp_listener.rs diff --git a/quickwit/quickwit-cli/src/service.rs b/quickwit/quickwit-cli/src/service.rs index 889c0455d5a..40693e9d4b5 100644 --- a/quickwit/quickwit-cli/src/service.rs +++ b/quickwit/quickwit-cli/src/service.rs @@ -27,6 +27,7 @@ use quickwit_common::runtimes::RuntimesConfig; use quickwit_common::uri::{Protocol, Uri}; use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; +use quickwit_serve::tcp_listener::DefaultTcpListenerResolver; use quickwit_serve::{serve_quickwit, BuildInfo, EnvFilterReloadFn}; use quickwit_telemetry::payload::{QuickwitFeature, QuickwitTelemetryInfo, TelemetryEvent}; use tokio::signal; @@ -114,6 +115,7 @@ impl RunCliCommand { runtimes_config, metastore_resolver, storage_resolver, + DefaultTcpListenerResolver, shutdown_signal, env_filter_reload_fn, ) diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index f8a5611d75e..2321bd0399a 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -480,9 +480,17 @@ impl NodeConfig { self.storage_configs.redact(); } + /// Creates a config with defaults suitable for testing. + /// + /// Uses the default ports without ensuring that they are available. #[cfg(any(test, feature = "testsuite"))] pub fn for_test() -> Self { - serialize::node_config_for_test() + serialize::node_config_for_tests_from_ports(7280, 7281) + } + + #[cfg(any(test, feature = "testsuite"))] + pub fn for_test_from_ports(rest_listen_port: u16, grpc_listen_port: u16) -> Self { + serialize::node_config_for_tests_from_ports(rest_listen_port, grpc_listen_port) } } diff --git a/quickwit/quickwit-config/src/node_config/serialize.rs b/quickwit/quickwit-config/src/node_config/serialize.rs index 208a929badc..81b9260f01d 100644 --- a/quickwit/quickwit-config/src/node_config/serialize.rs +++ b/quickwit/quickwit-config/src/node_config/serialize.rs @@ -411,13 +411,13 @@ impl RestConfigBuilder { } #[cfg(any(test, feature = "testsuite"))] -pub fn node_config_for_test() -> NodeConfig { - use quickwit_common::net::find_available_tcp_port; - +pub fn node_config_for_tests_from_ports( + rest_listen_port: u16, + grpc_listen_port: u16, +) -> NodeConfig { let node_id = NodeId::new(default_node_id().unwrap()); let enabled_services = QuickwitService::supported_services(); let listen_address = Host::default(); - let rest_listen_port = find_available_tcp_port().expect("OS should find an available port"); let rest_listen_addr = listen_address .with_port(rest_listen_port) .to_socket_addr() @@ -426,7 +426,6 @@ pub fn node_config_for_test() -> NodeConfig { .with_port(rest_listen_port) .to_socket_addr() .expect("default host should be an IP address"); - let grpc_listen_port = find_available_tcp_port().expect("OS should find an available port"); let grpc_listen_addr = listen_address .with_port(grpc_listen_port) .to_socket_addr() diff --git a/quickwit/quickwit-integration-tests/Cargo.toml b/quickwit/quickwit-integration-tests/Cargo.toml index ecfafd9b5a8..9a2892c3ad4 100644 --- a/quickwit/quickwit-integration-tests/Cargo.toml +++ b/quickwit/quickwit-integration-tests/Cargo.toml @@ -41,5 +41,5 @@ quickwit-metastore = { workspace = true, features = ["testsuite"] } quickwit-opentelemetry = { workspace = true, features = ["testsuite"] } quickwit-proto = { workspace = true, features = ["testsuite"] } quickwit-rest-client = { workspace = true } -quickwit-serve = { workspace = true } +quickwit-serve = { workspace = true, features = ["testsuite"] } quickwit-storage = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs index 2ccf38cc1a6..cfa86ab9e91 100644 --- a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs +++ b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs @@ -20,7 +20,6 @@ use std::collections::{HashMap, HashSet}; use std::io::Write; use std::net::SocketAddr; -use std::path::PathBuf; use std::str::FromStr; use std::time::Duration; @@ -31,8 +30,7 @@ use quickwit_actors::ActorExitStatus; use quickwit_cli::tool::{local_ingest_docs_cli, LocalIngestDocsArgs}; use quickwit_common::new_coolid; use quickwit_common::runtimes::RuntimesConfig; -use quickwit_common::test_utils::{wait_for_server_ready, wait_until_predicate}; -use quickwit_common::tower::BoxFutureInfaillible; +use quickwit_common::test_utils::wait_until_predicate; use quickwit_common::uri::Uri as QuickwitUri; use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; @@ -45,151 +43,150 @@ use quickwit_rest_client::models::IngestSource; use quickwit_rest_client::rest_client::{ CommitType, QuickwitClient, QuickwitClientBuilder, DEFAULT_BASE_URL, }; +use quickwit_serve::tcp_listener::for_tests::TestTcpListenerResolver; use quickwit_serve::{serve_quickwit, ListSplitsQueryParams, SearchRequestQueryString}; use quickwit_storage::StorageResolver; use reqwest::Url; use serde_json::Value; use tempfile::TempDir; -use tokio::sync::watch::{self, Receiver, Sender}; -use tokio::task::JoinHandle; +use tokio::net::TcpListener; use tonic::transport::channel; use tracing::debug; -/// Configuration of a node made of a [`NodeConfig`] and a -/// set of services. -#[derive(Clone)] +use super::shutdown::NodeShutdownHandle; + pub struct TestNodeConfig { - pub node_config: NodeConfig, pub services: HashSet, + pub enable_otlp: bool, } -type NodeJoinHandle = JoinHandle, anyhow::Error>>; - -struct NodeShutdownHandle { - sender: Sender<()>, - receiver: Receiver<()>, - node_services: HashSet, - node_id: NodeId, - join_handle_opt: Option, +pub struct ClusterSandboxBuilder { + temp_dir: TempDir, + node_configs: Vec, } -impl NodeShutdownHandle { - fn new(node_id: NodeId, node_services: HashSet) -> Self { - let (sender, receiver) = watch::channel(()); +impl Default for ClusterSandboxBuilder { + fn default() -> Self { Self { - sender, - receiver, - node_id, - node_services, - join_handle_opt: None, + temp_dir: tempfile::tempdir().unwrap(), + node_configs: Vec::new(), } } +} - fn shutdown_signal(&self) -> BoxFutureInfaillible<()> { - let receiver = self.receiver.clone(); - Box::pin(async move { - receiver.clone().changed().await.unwrap(); - }) +impl ClusterSandboxBuilder { + pub fn add_node(mut self, services: impl IntoIterator) -> Self { + self.node_configs.push(TestNodeConfig { + services: HashSet::from_iter(services), + enable_otlp: false, + }); + self } - fn set_node_join_handle(&mut self, join_handle: NodeJoinHandle) { - self.join_handle_opt = Some(join_handle); + pub fn add_node_with_otlp( + mut self, + services: impl IntoIterator, + ) -> Self { + self.node_configs.push(TestNodeConfig { + services: HashSet::from_iter(services), + enable_otlp: true, + }); + self } - /// Initiate node shutdown and wait for it to complete - async fn shutdown(self) -> anyhow::Result> { - self.sender.send(()).unwrap(); - self.join_handle_opt - .expect("node join handle was not set before shutdown") - .await - .unwrap() + /// Builds a list of of [`NodeConfig`] from the node definitions added to + /// builder. For each node, a [`NodeConfig`] is built with the right + /// parameters such that we will be able to run `quickwit_serve` on them and + /// form a Quickwit cluster. For each node, we set: + /// - `data_dir_path` defined by `root_data_dir/node_id`. + /// - `metastore_uri` defined by `root_data_dir/metastore`. + /// - `default_index_root_uri` defined by `root_data_dir/indexes`. + /// - `peers` defined by others nodes `gossip_advertise_addr`. + async fn build_config(self) -> ResolvedClusterConfig { + let root_data_dir = self.temp_dir.path().to_path_buf(); + let cluster_id = new_coolid("test-cluster"); + let mut resolved_node_configs = Vec::new(); + let mut peers: Vec = Vec::new(); + let unique_dir_name = new_coolid("test-dir"); + let tcp_listener_resolver = TestTcpListenerResolver::default(); + for (node_idx, node_builder) in self.node_configs.iter().enumerate() { + let socket: SocketAddr = ([127, 0, 0, 1], 0u16).into(); + let rest_tcp_listener = TcpListener::bind(socket).await.unwrap(); + let grpc_tcp_listener = TcpListener::bind(socket).await.unwrap(); + let mut config = NodeConfig::for_test_from_ports( + rest_tcp_listener.local_addr().unwrap().port(), + grpc_tcp_listener.local_addr().unwrap().port(), + ); + tcp_listener_resolver.add_listener(rest_tcp_listener).await; + tcp_listener_resolver.add_listener(grpc_tcp_listener).await; + config.indexer_config.enable_otlp_endpoint = node_builder.enable_otlp; + config.enabled_services.clone_from(&node_builder.services); + config.jaeger_config.enable_endpoint = true; + config.cluster_id.clone_from(&cluster_id); + config.node_id = NodeId::new(format!("test-node-{node_idx}")); + config.data_dir_path = root_data_dir.join(config.node_id.as_str()); + config.metastore_uri = + QuickwitUri::from_str(&format!("ram:///{unique_dir_name}/metastore")).unwrap(); + config.default_index_root_uri = + QuickwitUri::from_str(&format!("ram:///{unique_dir_name}/indexes")).unwrap(); + peers.push(config.gossip_advertise_addr.to_string()); + resolved_node_configs.push((config, node_builder.services.clone())); + } + for node_config in resolved_node_configs.iter_mut() { + node_config.0.peer_seeds = peers + .clone() + .into_iter() + .filter(|seed| *seed != node_config.0.gossip_advertise_addr.to_string()) + .collect_vec(); + } + ResolvedClusterConfig { + temp_dir: self.temp_dir, + node_configs: resolved_node_configs, + tcp_listener_resolver, + } } -} - -/// Creates a Cluster Test environment. -/// -/// The goal is to start several nodes and use the gRPC or REST clients to -/// test it. -/// -/// WARNING: Currently, we cannot start an indexer in a different test as it will -/// will share the same `INGEST_API_SERVICE_INSTANCE`. The ingest API will be -/// dropped by the first running test and the other tests will fail. -pub struct ClusterSandbox { - pub node_configs: Vec, - pub searcher_rest_client: QuickwitClient, - pub indexer_rest_client: QuickwitClient, - pub trace_client: TraceServiceClient, - pub logs_client: LogsServiceClient, - pub jaeger_client: SpanReaderPluginClient, - _temp_dir: TempDir, - node_shutdown_handle: Vec, -} -fn transport_url(addr: SocketAddr) -> Url { - let mut url = Url::parse(DEFAULT_BASE_URL).unwrap(); - url.set_ip_host(addr.ip()).unwrap(); - url.set_port(Some(addr.port())).unwrap(); - url -} + pub async fn build_and_start(self) -> ClusterSandbox { + self.build_config().await.start().await + } -#[macro_export] -macro_rules! ingest_json { - ($($json:tt)+) => { - quickwit_rest_client::models::IngestSource::Str(json!($($json)+).to_string()) - }; + pub async fn build_and_start_standalone() -> ClusterSandbox { + ClusterSandboxBuilder::default() + .add_node(QuickwitService::supported_services()) + .build_config() + .await + .start() + .await + } } -pub(crate) async fn ingest_with_retry( - client: &QuickwitClient, - index_id: &str, - ingest_source: IngestSource, - commit_type: CommitType, -) -> anyhow::Result<()> { - wait_until_predicate( - || { - let commit_type_clone = commit_type; - let ingest_source_clone = ingest_source.clone(); - async move { - // Index one record. - if let Err(err) = client - .ingest(index_id, ingest_source_clone, None, None, commit_type_clone) - .await - { - debug!(index=%index_id, err=%err, "failed to ingest"); - false - } else { - true - } - } - }, - Duration::from_secs(10), - Duration::from_millis(100), - ) - .await?; - Ok(()) +/// Intermediate state where the ports of all the the test cluster nodes have +/// been reserved and the configurations have been generated. +struct ResolvedClusterConfig { + temp_dir: TempDir, + node_configs: Vec<(NodeConfig, HashSet)>, + tcp_listener_resolver: TestTcpListenerResolver, } -impl ClusterSandbox { - pub async fn start_cluster_with_configs( - temp_dir: TempDir, - node_configs: Vec, - ) -> anyhow::Result { +impl ResolvedClusterConfig { + /// Start a cluster using this config and waits for the nodes to be ready + pub async fn start(self) -> ClusterSandbox { + let mut node_shutdown_handles = Vec::new(); let runtimes_config = RuntimesConfig::light_for_tests(); let storage_resolver = StorageResolver::unconfigured(); let metastore_resolver = MetastoreResolver::unconfigured(); - let mut node_shutdown_handlers = Vec::new(); - for node_config in node_configs.iter() { - let mut shutdown_handler = NodeShutdownHandle::new( - node_config.node_config.node_id.clone(), - node_config.services.clone(), - ); + let cluster_size = self.node_configs.len(); + for node_config in self.node_configs.iter() { + let mut shutdown_handler = + NodeShutdownHandle::new(node_config.0.node_id.clone(), node_config.1.clone()); let shutdown_signal = shutdown_handler.shutdown_signal(); let join_handle = tokio::spawn({ - let node_config = node_config.node_config.clone(); + let node_config = node_config.0.clone(); let node_id = node_config.node_id.clone(); let services = node_config.enabled_services.clone(); let metastore_resolver = metastore_resolver.clone(); let storage_resolver = storage_resolver.clone(); + let tcp_listener_resolver = self.tcp_listener_resolver.clone(); async move { let result = serve_quickwit( @@ -197,6 +194,7 @@ impl ClusterSandbox { runtimes_config, metastore_resolver, storage_resolver, + tcp_listener_resolver, shutdown_signal, quickwit_serve::do_nothing_env_filter_reload_fn(), ) @@ -206,92 +204,117 @@ impl ClusterSandbox { } }); shutdown_handler.set_node_join_handle(join_handle); - node_shutdown_handlers.push(shutdown_handler); + node_shutdown_handles.push(shutdown_handler); } - let searcher_config = node_configs + let searcher_config = self + .node_configs .iter() - .find(|node_config| node_config.services.contains(&QuickwitService::Searcher)) + .find(|node_config| node_config.1.contains(&QuickwitService::Searcher)) .cloned() .unwrap(); - let indexer_config = node_configs + let indexer_config = self + .node_configs .iter() - .find(|node_config| node_config.services.contains(&QuickwitService::Indexer)) + .find(|node_config| node_config.1.contains(&QuickwitService::Indexer)) .cloned() .unwrap(); - if node_configs.len() == 1 { - // We have only one node, so we can just wait for it to get started - wait_for_server_ready(node_configs[0].node_config.grpc_listen_addr).await?; - } else { - // Wait for a duration greater than chitchat GOSSIP_INTERVAL (50ms) so that the cluster - // is formed. - tokio::time::sleep(Duration::from_millis(100)).await; - } - let indexer_channel = channel::Endpoint::from_str(&format!( - "http://{}", - indexer_config.node_config.grpc_listen_addr - )) - .unwrap() - .connect_lazy(); - let searcher_channel = channel::Endpoint::from_str(&format!( - "http://{}", - searcher_config.node_config.grpc_listen_addr - )) - .unwrap() - .connect_lazy(); - Ok(Self { - node_configs, + let indexer_channel = + channel::Endpoint::from_str(&format!("http://{}", indexer_config.0.grpc_listen_addr)) + .unwrap() + .connect_lazy(); + let searcher_channel = + channel::Endpoint::from_str(&format!("http://{}", searcher_config.0.grpc_listen_addr)) + .unwrap() + .connect_lazy(); + + let sandbox = ClusterSandbox { + node_configs: self.node_configs, searcher_rest_client: QuickwitClientBuilder::new(transport_url( - searcher_config.node_config.rest_config.listen_addr, + searcher_config.0.rest_config.listen_addr, )) .build(), indexer_rest_client: QuickwitClientBuilder::new(transport_url( - indexer_config.node_config.rest_config.listen_addr, + indexer_config.0.rest_config.listen_addr, )) .build(), trace_client: TraceServiceClient::new(indexer_channel.clone()), logs_client: LogsServiceClient::new(indexer_channel), jaeger_client: SpanReaderPluginClient::new(searcher_channel), - _temp_dir: temp_dir, - node_shutdown_handle: node_shutdown_handlers, - }) + _temp_dir: self.temp_dir, + node_shutdown_handles, + }; + sandbox + .wait_for_cluster_num_ready_nodes(cluster_size) + .await + .unwrap(); + sandbox } +} - pub fn enable_ingest_v2(&mut self) { - self.indexer_rest_client.enable_ingest_v2(); - self.searcher_rest_client.enable_ingest_v2(); - } +fn transport_url(addr: SocketAddr) -> Url { + let mut url = Url::parse(DEFAULT_BASE_URL).unwrap(); + url.set_ip_host(addr.ip()).unwrap(); + url.set_port(Some(addr.port())).unwrap(); + url +} - // Starts one node that runs all the services and wait for it to be ready - pub async fn start_standalone_node() -> anyhow::Result { - let sandbox = Self::start_cluster_nodes(&[QuickwitService::supported_services()]).await?; - sandbox.wait_for_cluster_num_ready_nodes(1).await?; - Ok(sandbox) - } +#[macro_export] +macro_rules! ingest_json { + ($($json:tt)+) => { + quickwit_rest_client::models::IngestSource::Str(json!($($json)+).to_string()) + }; +} - pub async fn start_cluster_with_otlp_service( - nodes_services: &[HashSet], - ) -> anyhow::Result { - let temp_dir = tempfile::tempdir()?; - let mut node_configs = build_node_configs(temp_dir.path().to_path_buf(), nodes_services); - // Set OTLP endpoint for indexers. - for node_config in node_configs.iter_mut() { - if node_config.services.contains(&QuickwitService::Indexer) { - node_config.node_config.indexer_config.enable_otlp_endpoint = true; +pub(crate) async fn ingest_with_retry( + client: &QuickwitClient, + index_id: &str, + ingest_source: IngestSource, + commit_type: CommitType, +) -> anyhow::Result<()> { + wait_until_predicate( + || { + let commit_type_clone = commit_type; + let ingest_source_clone = ingest_source.clone(); + async move { + // Index one record. + if let Err(err) = client + .ingest(index_id, ingest_source_clone, None, None, commit_type_clone) + .await + { + debug!(index=%index_id, err=%err, "failed to ingest"); + false + } else { + true + } } - } - Self::start_cluster_with_configs(temp_dir, node_configs).await - } + }, + Duration::from_secs(10), + Duration::from_millis(100), + ) + .await?; + Ok(()) +} - // Starts nodes with corresponding services given by `nodes_services`. - pub async fn start_cluster_nodes( - nodes_services: &[HashSet], - ) -> anyhow::Result { - let temp_dir = tempfile::tempdir()?; - let node_configs = build_node_configs(temp_dir.path().to_path_buf(), nodes_services); - Self::start_cluster_with_configs(temp_dir, node_configs).await +/// A test environment where you can start a Quickwit cluster and use the gRPC +/// or REST clients to test it. +pub struct ClusterSandbox { + pub node_configs: Vec<(NodeConfig, HashSet)>, + pub searcher_rest_client: QuickwitClient, + pub indexer_rest_client: QuickwitClient, + pub trace_client: TraceServiceClient, + pub logs_client: LogsServiceClient, + pub jaeger_client: SpanReaderPluginClient, + _temp_dir: TempDir, + node_shutdown_handles: Vec, +} + +impl ClusterSandbox { + pub fn enable_ingest_v2(&mut self) { + self.indexer_rest_client.enable_ingest_v2(); + self.searcher_rest_client.enable_ingest_v2(); } - pub async fn wait_for_cluster_num_ready_nodes( + async fn wait_for_cluster_num_ready_nodes( &self, expected_num_ready_nodes: usize, ) -> anyhow::Result<()> { @@ -394,7 +417,7 @@ impl ClusterSandbox { } } }, - Duration::from_secs(10), + Duration::from_secs(15), Duration::from_millis(500), ) .await?; @@ -405,7 +428,7 @@ impl ClusterSandbox { let test_conf = self .node_configs .iter() - .find(|config| config.services.contains(&QuickwitService::Indexer)) + .find(|config| config.1.contains(&QuickwitService::Indexer)) .ok_or(anyhow::anyhow!("No indexer node found"))?; // NodeConfig cannot be serialized, we write our own simplified config let mut tmp_config_file = tempfile::Builder::new().suffix(".yaml").tempfile().unwrap(); @@ -415,7 +438,7 @@ impl ClusterSandbox { metastore_uri: {} data_dir: {:?} "#, - test_conf.node_config.metastore_uri, test_conf.node_config.data_dir_path + test_conf.0.metastore_uri, test_conf.0.data_dir_path ); tmp_config_file.write_all(node_config.as_bytes())?; tmp_config_file.flush()?; @@ -479,10 +502,10 @@ impl ClusterSandbox { let mut shutdown_futures = Vec::new(); let mut shutdown_nodes = HashMap::new(); let mut i = 0; - while i < self.node_shutdown_handle.len() { - let handler_services = &self.node_shutdown_handle[i].node_services; + while i < self.node_shutdown_handles.len() { + let handler_services = &self.node_shutdown_handles[i].node_services; if handler_services.is_subset(shutdown_services) { - let handler_to_shutdown = self.node_shutdown_handle.remove(i); + let handler_to_shutdown = self.node_shutdown_handles.remove(i); shutdown_nodes.insert( handler_to_shutdown.node_id.clone(), handler_to_shutdown.node_services.clone(), @@ -508,48 +531,3 @@ impl ClusterSandbox { .await } } - -/// Builds a list of [`NodeConfig`] given a list of Quickwit services. -/// Each element of `nodes_services` defines the services of a given node. -/// For each node, a `NodeConfig` is built with the right parameters -/// such that we will be able to run `quickwit_serve` on them and form -/// a quickwit cluster. -/// For each node, we set: -/// - `data_dir_path` defined by `root_data_dir/node_id`. -/// - `metastore_uri` defined by `root_data_dir/metastore`. -/// - `default_index_root_uri` defined by `root_data_dir/indexes`. -/// - `peers` defined by others nodes `gossip_advertise_addr`. -pub fn build_node_configs( - root_data_dir: PathBuf, - nodes_services: &[HashSet], -) -> Vec { - let cluster_id = new_coolid("test-cluster"); - let mut node_configs = Vec::new(); - let mut peers: Vec = Vec::new(); - let unique_dir_name = new_coolid("test-dir"); - for (node_idx, node_services) in nodes_services.iter().enumerate() { - let mut config = NodeConfig::for_test(); - config.enabled_services.clone_from(node_services); - config.jaeger_config.enable_endpoint = true; - config.cluster_id.clone_from(&cluster_id); - config.node_id = NodeId::new(format!("test-node-{node_idx}")); - config.data_dir_path = root_data_dir.join(config.node_id.as_str()); - config.metastore_uri = - QuickwitUri::from_str(&format!("ram:///{unique_dir_name}/metastore")).unwrap(); - config.default_index_root_uri = - QuickwitUri::from_str(&format!("ram:///{unique_dir_name}/indexes")).unwrap(); - peers.push(config.gossip_advertise_addr.to_string()); - node_configs.push(TestNodeConfig { - node_config: config, - services: node_services.clone(), - }); - } - for node_config in node_configs.iter_mut() { - node_config.node_config.peer_seeds = peers - .clone() - .into_iter() - .filter(|seed| *seed != node_config.node_config.gossip_advertise_addr.to_string()) - .collect_vec(); - } - node_configs -} diff --git a/quickwit/quickwit-integration-tests/src/test_utils/mod.rs b/quickwit/quickwit-integration-tests/src/test_utils/mod.rs index 82d2213068f..00bbdf73e82 100644 --- a/quickwit/quickwit-integration-tests/src/test_utils/mod.rs +++ b/quickwit/quickwit-integration-tests/src/test_utils/mod.rs @@ -18,5 +18,6 @@ // along with this program. If not, see . mod cluster_sandbox; +mod shutdown; -pub(crate) use cluster_sandbox::{ingest_with_retry, ClusterSandbox}; +pub(crate) use cluster_sandbox::{ingest_with_retry, ClusterSandbox, ClusterSandboxBuilder}; diff --git a/quickwit/quickwit-integration-tests/src/test_utils/shutdown.rs b/quickwit/quickwit-integration-tests/src/test_utils/shutdown.rs new file mode 100644 index 00000000000..0e9c1aa4e38 --- /dev/null +++ b/quickwit/quickwit-integration-tests/src/test_utils/shutdown.rs @@ -0,0 +1,73 @@ +// 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::{HashMap, HashSet}; + +use quickwit_actors::ActorExitStatus; +use quickwit_common::tower::BoxFutureInfaillible; +use quickwit_config::service::QuickwitService; +use quickwit_proto::types::NodeId; +use tokio::sync::watch::{self, Receiver, Sender}; +use tokio::task::JoinHandle; + +type NodeJoinHandle = JoinHandle, anyhow::Error>>; + +pub(crate) struct NodeShutdownHandle { + sender: Sender<()>, + receiver: Receiver<()>, + pub node_services: HashSet, + pub node_id: NodeId, + join_handle_opt: Option, +} + +impl NodeShutdownHandle { + pub(crate) fn new(node_id: NodeId, node_services: HashSet) -> Self { + let (sender, receiver) = watch::channel(()); + Self { + sender, + receiver, + node_id, + node_services, + join_handle_opt: None, + } + } + + pub(crate) fn shutdown_signal(&self) -> BoxFutureInfaillible<()> { + let receiver = self.receiver.clone(); + Box::pin(async move { + receiver.clone().changed().await.unwrap(); + }) + } + + pub(crate) fn set_node_join_handle(&mut self, join_handle: NodeJoinHandle) { + self.join_handle_opt = Some(join_handle); + } + + /// Initiate node shutdown and wait for it to complete + + pub(crate) async fn shutdown( + self, + ) -> anyhow::Result> { + self.sender.send(()).unwrap(); + self.join_handle_opt + .expect("node join handle was not set before shutdown") + .await + .unwrap() + } +} diff --git a/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs b/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs index ee6f18eafe0..66b71c6738b 100644 --- a/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs @@ -17,7 +17,6 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::HashSet; use std::path::PathBuf; use std::str::FromStr; use std::time::Duration; @@ -28,7 +27,7 @@ use quickwit_rest_client::models::IngestSource; use quickwit_rest_client::rest_client::CommitType; use quickwit_serve::SearchRequestQueryString; -use crate::test_utils::{ingest_with_retry, ClusterSandbox}; +use crate::test_utils::{ingest_with_retry, ClusterSandboxBuilder}; fn get_ndjson_filepath(ndjson_dataset_filename: &str) -> String { format!( @@ -41,18 +40,15 @@ fn get_ndjson_filepath(ndjson_dataset_filename: &str) -> String { #[tokio::test] async fn test_ui_redirect_on_get() { quickwit_common::setup_logging_for_tests(); - let sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; let node_config = sandbox.node_configs.first().unwrap(); let client = hyper::Client::builder() .pool_idle_timeout(Duration::from_secs(30)) .http2_only(true) .build_http(); - let root_uri = format!( - "http://{}/", - node_config.node_config.rest_config.listen_addr - ) - .parse::() - .unwrap(); + let root_uri = format!("http://{}/", node_config.0.rest_config.listen_addr) + .parse::() + .unwrap(); let response = client.get(root_uri.clone()).await.unwrap(); assert_eq!(response.status(), StatusCode::MOVED_PERMANENTLY); let post_request = Request::builder() @@ -68,7 +64,7 @@ async fn test_ui_redirect_on_get() { #[tokio::test] async fn test_standalone_server() { quickwit_common::setup_logging_for_tests(); - let sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; { // The indexing service should be running. let counters = sandbox @@ -125,17 +121,14 @@ async fn test_standalone_server() { #[tokio::test] async fn test_multi_nodes_cluster() { quickwit_common::setup_logging_for_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Searcher]), - HashSet::from_iter([QuickwitService::Metastore]), - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([QuickwitService::ControlPlane]), - HashSet::from_iter([QuickwitService::Janitor]), - ]; - let sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services) - .await - .unwrap(); - sandbox.wait_for_cluster_num_ready_nodes(5).await.unwrap(); + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; { // Wait for indexer to fully start. diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index b4c3eab3325..e109facf105 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -17,7 +17,6 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::HashSet; use std::time::Duration; use hyper::StatusCode; @@ -29,7 +28,7 @@ use quickwit_rest_client::rest_client::CommitType; use serde_json::json; use crate::ingest_json; -use crate::test_utils::{ingest_with_retry, ClusterSandbox}; +use crate::test_utils::{ingest_with_retry, ClusterSandboxBuilder}; fn initialize_tests() { quickwit_common::setup_logging_for_tests(); @@ -39,7 +38,7 @@ fn initialize_tests() { #[tokio::test] async fn test_single_node_cluster() { initialize_tests(); - let mut sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; let index_id = "test-single-node-cluster"; let index_config = format!( r#" @@ -59,7 +58,6 @@ async fn test_single_node_cluster() { index_id ); sandbox.enable_ingest_v2(); - sandbox.wait_for_cluster_num_ready_nodes(1).await.unwrap(); // Create the index. let current_index_metadata = sandbox @@ -203,20 +201,17 @@ async fn test_single_node_cluster() { #[tokio::test] async fn test_ingest_v2_index_not_found() { initialize_tests(); - let nodes_services = &[ - HashSet::from_iter([QuickwitService::Indexer, QuickwitService::Janitor]), - HashSet::from_iter([QuickwitService::Indexer, QuickwitService::Janitor]), - HashSet::from_iter([ + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Indexer, QuickwitService::Janitor]) + .add_node([QuickwitService::Indexer, QuickwitService::Janitor]) + .add_node([ QuickwitService::ControlPlane, QuickwitService::Metastore, QuickwitService::Searcher, - ]), - ]; - let mut sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services[..]) - .await - .unwrap(); + ]) + .build_and_start() + .await; sandbox.enable_ingest_v2(); - sandbox.wait_for_cluster_num_ready_nodes(3).await.unwrap(); let missing_index_err: Error = sandbox .indexer_rest_client .ingest( @@ -241,21 +236,17 @@ async fn test_ingest_v2_index_not_found() { #[tokio::test] async fn test_ingest_v2_happy_path() { initialize_tests(); - - let nodes_services = &[ - HashSet::from_iter([QuickwitService::Indexer, QuickwitService::Janitor]), - HashSet::from_iter([QuickwitService::Indexer, QuickwitService::Janitor]), - HashSet::from_iter([ + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Indexer, QuickwitService::Janitor]) + .add_node([QuickwitService::Indexer, QuickwitService::Janitor]) + .add_node([ QuickwitService::ControlPlane, QuickwitService::Metastore, QuickwitService::Searcher, - ]), - ]; - let mut sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services[..]) - .await - .unwrap(); + ]) + .build_and_start() + .await; sandbox.enable_ingest_v2(); - sandbox.wait_for_cluster_num_ready_nodes(3).await.unwrap(); let index_id = "test_happy_path"; let index_config = format!( r#" @@ -328,7 +319,7 @@ async fn test_ingest_v2_happy_path() { #[tokio::test] async fn test_commit_modes() { initialize_tests(); - let sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; let index_id = "test_commit_modes"; let index_config = format!( r#" @@ -396,7 +387,10 @@ async fn test_commit_modes() { sandbox.assert_hit_count(index_id, "body:auto", 0).await; - tokio::time::sleep(Duration::from_secs(3)).await; + sandbox + .wait_for_splits(index_id, Some(vec![SplitState::Published]), 3) + .await + .unwrap(); sandbox.assert_hit_count(index_id, "body:auto", 1).await; @@ -407,18 +401,15 @@ async fn test_commit_modes() { #[tokio::test] async fn test_very_large_index_name() { initialize_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Searcher]), - HashSet::from_iter([QuickwitService::Metastore]), - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([QuickwitService::ControlPlane]), - HashSet::from_iter([QuickwitService::Janitor]), - ]; - let mut sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services) - .await - .unwrap(); + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; sandbox.enable_ingest_v2(); - sandbox.wait_for_cluster_num_ready_nodes(5).await.unwrap(); let index_id = "its_very_very_very_very_very_very_very_very_very_very_very_\ very_very_very_very_very_very_very_very_very_very_very_very_very_very_very_\ @@ -509,7 +500,7 @@ async fn test_very_large_index_name() { #[tokio::test] async fn test_shutdown_single_node() { initialize_tests(); - let mut sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; let index_id = "test_shutdown_single_node"; sandbox.enable_ingest_v2(); @@ -571,18 +562,16 @@ async fn test_shutdown_single_node() { #[tokio::test] async fn test_shutdown_control_plane_early_shutdown() { initialize_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([ + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Indexer]) + .add_node([ QuickwitService::ControlPlane, QuickwitService::Searcher, QuickwitService::Metastore, QuickwitService::Janitor, - ]), - ]; - let sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services) - .await - .unwrap(); + ]) + .build_and_start() + .await; let index_id = "test_shutdown_separate_indexer"; // TODO: make this test work with ingest v2 (#5068) @@ -632,18 +621,16 @@ async fn test_shutdown_control_plane_early_shutdown() { #[tokio::test] async fn test_shutdown_separate_indexer() { initialize_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([ + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Indexer]) + .add_node([ QuickwitService::ControlPlane, QuickwitService::Searcher, QuickwitService::Metastore, QuickwitService::Janitor, - ]), - ]; - let sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services) - .await - .unwrap(); + ]) + .build_and_start() + .await; let index_id = "test_shutdown_separate_indexer"; // TODO: make this test work with ingest v2 (#5068) diff --git a/quickwit/quickwit-integration-tests/src/tests/otlp_tests.rs b/quickwit/quickwit-integration-tests/src/tests/otlp_tests.rs index 752fe45acf3..31df6858fd6 100644 --- a/quickwit/quickwit-integration-tests/src/tests/otlp_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/otlp_tests.rs @@ -37,7 +37,7 @@ use quickwit_proto::opentelemetry::proto::logs::v1::{LogRecord, ResourceLogs, Sc use quickwit_proto::opentelemetry::proto::trace::v1::{ResourceSpans, ScopeSpans, Span}; use tonic::codec::CompressionEncoding; -use crate::test_utils::ClusterSandbox; +use crate::test_utils::ClusterSandboxBuilder; fn initialize_tests() { quickwit_common::setup_logging_for_tests(); @@ -47,16 +47,14 @@ fn initialize_tests() { #[tokio::test] async fn test_ingest_traces_with_otlp_grpc_api() { initialize_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Searcher]), - HashSet::from_iter([QuickwitService::Metastore]), - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([QuickwitService::ControlPlane]), - HashSet::from_iter([QuickwitService::Janitor]), - ]; - let mut sandbox = ClusterSandbox::start_cluster_with_otlp_service(&nodes_services) - .await - .unwrap(); + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node_with_otlp([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; // Wait for the pipelines to start (one for logs and one for traces) sandbox.wait_for_indexing_pipelines(2).await.unwrap(); @@ -142,16 +140,14 @@ async fn test_ingest_traces_with_otlp_grpc_api() { #[tokio::test] async fn test_ingest_logs_with_otlp_grpc_api() { initialize_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Searcher]), - HashSet::from_iter([QuickwitService::Metastore]), - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([QuickwitService::ControlPlane]), - HashSet::from_iter([QuickwitService::Janitor]), - ]; - let mut sandbox = ClusterSandbox::start_cluster_with_otlp_service(&nodes_services) - .await - .unwrap(); + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node_with_otlp([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; // Wait fo the pipelines to start (one for logs and one for traces) sandbox.wait_for_indexing_pipelines(2).await.unwrap(); @@ -218,16 +214,14 @@ async fn test_ingest_logs_with_otlp_grpc_api() { #[tokio::test] async fn test_jaeger_api() { initialize_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Searcher]), - HashSet::from_iter([QuickwitService::Metastore]), - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([QuickwitService::ControlPlane]), - HashSet::from_iter([QuickwitService::Janitor]), - ]; - let mut sandbox = ClusterSandbox::start_cluster_with_otlp_service(&nodes_services) - .await - .unwrap(); + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node_with_otlp([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; // Wait fo the pipelines to start (one for logs and one for traces) sandbox.wait_for_indexing_pipelines(2).await.unwrap(); diff --git a/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs b/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs index 419b1f04f2f..9ae9bd90e21 100644 --- a/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs @@ -32,7 +32,7 @@ use quickwit_serve::SearchRequestQueryString; use tempfile::NamedTempFile; use tracing::info; -use crate::test_utils::ClusterSandbox; +use crate::test_utils::ClusterSandboxBuilder; fn create_mock_data_file(num_lines: usize) -> (NamedTempFile, Uri) { let mut temp_file = tempfile::NamedTempFile::new().unwrap(); @@ -48,7 +48,7 @@ fn create_mock_data_file(num_lines: usize) -> (NamedTempFile, Uri) { #[tokio::test] async fn test_sqs_single_node_cluster() { quickwit_common::setup_logging_for_tests(); - let sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; let index_id = "test-sqs-source-single-node-cluster"; let index_config = format!( r#" @@ -68,8 +68,6 @@ async fn test_sqs_single_node_cluster() { let sqs_client = sqs_test_helpers::get_localstack_sqs_client().await.unwrap(); let queue_url = sqs_test_helpers::create_queue(&sqs_client, "test-single-node-cluster").await; - sandbox.wait_for_cluster_num_ready_nodes(1).await.unwrap(); - info!("create index"); sandbox .indexer_rest_client diff --git a/quickwit/quickwit-integration-tests/src/tests/update_tests/doc_mapping_tests.rs b/quickwit/quickwit-integration-tests/src/tests/update_tests/doc_mapping_tests.rs index cf44ed8bc54..c1d55860b20 100644 --- a/quickwit/quickwit-integration-tests/src/tests/update_tests/doc_mapping_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/update_tests/doc_mapping_tests.rs @@ -22,7 +22,7 @@ use std::time::Duration; use serde_json::{json, Value}; use super::assert_hits_unordered; -use crate::test_utils::ClusterSandbox; +use crate::test_utils::ClusterSandboxBuilder; /// Update the doc mapping between 2 calls to local-ingest (forces separate indexing pipelines) and /// assert the number of hits for the given query @@ -35,7 +35,7 @@ async fn validate_search_across_doc_mapping_updates( query_and_expect: &[(&str, Result<&[Value], ()>)], ) { quickwit_common::setup_logging_for_tests(); - let sandbox = ClusterSandbox::start_standalone_node().await.unwrap(); + let sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; { // Wait for indexer to fully start. diff --git a/quickwit/quickwit-integration-tests/src/tests/update_tests/search_settings_tests.rs b/quickwit/quickwit-integration-tests/src/tests/update_tests/search_settings_tests.rs index 52d43627f22..9da970fbb98 100644 --- a/quickwit/quickwit-integration-tests/src/tests/update_tests/search_settings_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/update_tests/search_settings_tests.rs @@ -17,7 +17,6 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::HashSet; use std::time::Duration; use quickwit_config::service::QuickwitService; @@ -26,22 +25,19 @@ use serde_json::json; use super::assert_hits_unordered; use crate::ingest_json; -use crate::test_utils::{ingest_with_retry, ClusterSandbox}; +use crate::test_utils::{ingest_with_retry, ClusterSandboxBuilder}; #[tokio::test] async fn test_update_search_settings_on_multi_nodes_cluster() { quickwit_common::setup_logging_for_tests(); - let nodes_services = vec![ - HashSet::from_iter([QuickwitService::Searcher]), - HashSet::from_iter([QuickwitService::Metastore]), - HashSet::from_iter([QuickwitService::Indexer]), - HashSet::from_iter([QuickwitService::ControlPlane]), - HashSet::from_iter([QuickwitService::Janitor]), - ]; - let sandbox = ClusterSandbox::start_cluster_nodes(&nodes_services) - .await - .unwrap(); - sandbox.wait_for_cluster_num_ready_nodes(5).await.unwrap(); + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; { // Wait for indexer to fully start. diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index 22db2523d05..b82db775761 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -97,4 +97,4 @@ quickwit-storage = { workspace = true, features = ["testsuite"] } pprof = [ "dep:pprof" ] - +testsuite = [] diff --git a/quickwit/quickwit-serve/src/grpc.rs b/quickwit/quickwit-serve/src/grpc.rs index 7a1c24691ee..403ae46d853 100644 --- a/quickwit/quickwit-serve/src/grpc.rs +++ b/quickwit/quickwit-serve/src/grpc.rs @@ -18,7 +18,7 @@ // along with this program. If not, see . use std::collections::BTreeSet; -use std::net::SocketAddr; +use std::error::Error; use std::sync::Arc; use bytesize::ByteSize; @@ -32,7 +32,9 @@ use quickwit_proto::opentelemetry::proto::collector::logs::v1::logs_service_serv use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_server::TraceServiceServer; use quickwit_proto::search::search_service_server::SearchServiceServer; use quickwit_proto::tonic::codegen::CompressionEncoding; +use quickwit_proto::tonic::transport::server::TcpIncoming; use quickwit_proto::tonic::transport::Server; +use tokio::net::TcpListener; use tracing::*; use crate::developer_api::DeveloperApiServer; @@ -41,7 +43,7 @@ use crate::{QuickwitServices, INDEXING_GRPC_SERVER_METRICS_LAYER}; /// Starts and binds gRPC services to `grpc_listen_addr`. pub(crate) async fn start_grpc_server( - grpc_listen_addr: SocketAddr, + tcp_listener: TcpListener, max_message_size: ByteSize, services: Arc, readiness_trigger: BoxFutureInfaillible<()>, @@ -186,12 +188,16 @@ pub(crate) async fn start_grpc_server( .add_optional_service(otlp_trace_grpc_service) .add_optional_service(search_grpc_service); + let grpc_listen_addr = tcp_listener.local_addr()?; info!( enabled_grpc_services=?enabled_grpc_services, grpc_listen_addr=?grpc_listen_addr, "Starting gRPC server listening on {grpc_listen_addr}." ); - let serve_fut = server_router.serve_with_shutdown(grpc_listen_addr, shutdown_signal); + // nodelay=true and keepalive=None are the default values for Server::builder() + let tcp_incoming = TcpIncoming::from_listener(tcp_listener, true, None) + .map_err(|err: Box| anyhow::anyhow!(err))?; + let serve_fut = server_router.serve_with_incoming_shutdown(tcp_incoming, shutdown_signal); let (serve_res, _trigger_res) = tokio::join!(serve_fut, readiness_trigger); serve_res?; Ok(()) diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 2b5b7f123f9..4de79fe5d1b 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -43,6 +43,7 @@ mod rest; mod rest_api_response; mod search_api; pub(crate) mod simple_list; +pub mod tcp_listener; mod template_api; mod ui_handler; @@ -115,6 +116,7 @@ use quickwit_search::{ SearchServiceClient, SearcherContext, SearcherPool, }; use quickwit_storage::{SplitCache, StorageResolver}; +use tcp_listener::TcpListenerResolver; use tokio::sync::oneshot; use tower::timeout::Timeout; use tower::ServiceBuilder; @@ -387,6 +389,7 @@ pub async fn serve_quickwit( runtimes_config: RuntimesConfig, metastore_resolver: MetastoreResolver, storage_resolver: StorageResolver, + tcp_listener_resolver: impl TcpListenerResolver, shutdown_signal: BoxFutureInfaillible<()>, env_filter_reload_fn: EnvFilterReloadFn, ) -> anyhow::Result> { @@ -712,7 +715,7 @@ pub async fn serve_quickwit( } }); let grpc_server = grpc::start_grpc_server( - grpc_listen_addr, + tcp_listener_resolver.resolve(grpc_listen_addr).await?, grpc_config.max_message_size, quickwit_services.clone(), grpc_readiness_trigger, @@ -732,7 +735,7 @@ pub async fn serve_quickwit( } }); let rest_server = rest::start_rest_server( - rest_listen_addr, + tcp_listener_resolver.resolve(rest_listen_addr).await?, quickwit_services, rest_readiness_trigger, rest_shutdown_signal, diff --git a/quickwit/quickwit-serve/src/rest.rs b/quickwit/quickwit-serve/src/rest.rs index 483961a7378..a021da9c6f7 100644 --- a/quickwit/quickwit-serve/src/rest.rs +++ b/quickwit/quickwit-serve/src/rest.rs @@ -18,13 +18,13 @@ // along with this program. If not, see . use std::fmt::Formatter; -use std::net::SocketAddr; use std::sync::Arc; use hyper::http::HeaderValue; use hyper::{http, Method, StatusCode}; use quickwit_common::tower::BoxFutureInfaillible; use quickwit_search::SearchService; +use tokio::net::TcpListener; use tower::make::Shared; use tower::ServiceBuilder; use tower_http::compression::predicate::{NotForContentType, Predicate, SizeAbove}; @@ -123,7 +123,7 @@ impl Predicate for CompressionPredicate { /// Starts REST services. pub(crate) async fn start_rest_server( - rest_listen_addr: SocketAddr, + tcp_listener: TcpListener, quickwit_services: Arc, readiness_trigger: BoxFutureInfaillible<()>, shutdown_signal: BoxFutureInfaillible<()>, @@ -209,24 +209,26 @@ pub(crate) async fn start_rest_server( .layer(cors) .service(warp_service); + let rest_listen_addr = tcp_listener.local_addr()?; info!( rest_listen_addr=?rest_listen_addr, "Starting REST server listening on {rest_listen_addr}." ); + let rest_listener_std = tcp_listener.into_std()?; // `graceful_shutdown()` seems to be blocking in presence of existing connections. // The following approach of dropping the serve supposedly is not bullet proof, but it seems to // work in our unit test. // // See more of the discussion here: // https://github.com/hyperium/hyper/issues/2386 + let serve_fut = async move { tokio::select! { - res = hyper::Server::bind(&rest_listen_addr).serve(Shared::new(service)) => { res } + res = hyper::Server::from_tcp(rest_listener_std)?.serve(Shared::new(service)) => { res } _ = shutdown_signal => { Ok(()) } } }; - let (serve_res, _trigger_res) = tokio::join!(serve_fut, readiness_trigger); serve_res?; Ok(()) diff --git a/quickwit/quickwit-serve/src/tcp_listener.rs b/quickwit/quickwit-serve/src/tcp_listener.rs new file mode 100644 index 00000000000..749690c648d --- /dev/null +++ b/quickwit/quickwit-serve/src/tcp_listener.rs @@ -0,0 +1,81 @@ +// 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::net::SocketAddr; + +use quickwit_proto::tonic; +use tokio::net::TcpListener; +use tonic::async_trait; + +/// Resolve `SocketAddr` into `TcpListener` instances. +/// +/// This trait can be used to inject existing [`TcpListener`] instances to the +/// Quickwit REST and gRPC servers when running them in tests. +#[async_trait] +pub trait TcpListenerResolver: Clone + Send + 'static { + async fn resolve(&self, addr: SocketAddr) -> anyhow::Result; +} + +#[derive(Clone)] +pub struct DefaultTcpListenerResolver; + +#[async_trait] +impl TcpListenerResolver for DefaultTcpListenerResolver { + async fn resolve(&self, addr: SocketAddr) -> anyhow::Result { + TcpListener::bind(addr) + .await + .map_err(|err| anyhow::anyhow!(err)) + } +} + +#[cfg(any(test, feature = "testsuite"))] +pub mod for_tests { + use std::collections::HashMap; + use std::sync::Arc; + + use anyhow::Context; + use tokio::sync::Mutex; + + use super::*; + + #[derive(Clone, Default)] + pub struct TestTcpListenerResolver { + listeners: Arc>>, + } + + #[async_trait] + impl TcpListenerResolver for TestTcpListenerResolver { + async fn resolve(&self, addr: SocketAddr) -> anyhow::Result { + self.listeners + .lock() + .await + .remove(&addr) + .context(format!("No listener found for address {}", addr)) + } + } + + impl TestTcpListenerResolver { + pub async fn add_listener(&self, listener: TcpListener) { + self.listeners + .lock() + .await + .insert(listener.local_addr().unwrap(), listener); + } + } +} From 7cca6e42510969acdc843b5d9322d76b46d88e36 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:18 +0200 Subject: [PATCH 04/25] Add unit test to show force commit doesn't work --- .../src/tests/ingest_tests.rs | 100 ++++++++++++++---- 1 file changed, 81 insertions(+), 19 deletions(-) diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index e109facf105..3412b35dbbb 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -317,10 +317,10 @@ async fn test_ingest_v2_happy_path() { } #[tokio::test] -async fn test_commit_modes() { +async fn test_commit_force() { initialize_tests(); - let sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; - let index_id = "test_commit_modes"; + let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; + let index_id = "test_commit_force"; let index_config = format!( r#" version: 0.8 @@ -330,7 +330,7 @@ async fn test_commit_modes() { - name: body type: text indexing_settings: - commit_timeout_secs: 2 + commit_timeout_secs: 20 "# ); @@ -342,10 +342,8 @@ async fn test_commit_modes() { .await .unwrap(); - // TODO: make this test work with ingest v2 (#4438) - // sandbox.enable_ingest_v2(); + sandbox.enable_ingest_v2(); - // Test force commit ingest_with_retry( &sandbox.indexer_rest_client, index_id, @@ -355,24 +353,89 @@ async fn test_commit_modes() { .await .unwrap(); + // commit_timeout_secs is set to a large value, so this will timeout if + // CommitType::Force is not working + sandbox + .wait_for_splits(index_id, Some(vec![SplitState::Published]), 1) + .await + .unwrap(); + sandbox.assert_hit_count(index_id, "body:force", 1).await; - // Test wait_for commit + sandbox.shutdown().await.unwrap(); +} + +// #[tokio::test] +// async fn test_commit_wait_for() { +// initialize_tests(); +// let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; +// let index_id = "test_commit_wait_for"; +// let index_config = format!( +// r#" +// version: 0.8 +// index_id: {index_id} +// doc_mapping: +// field_mappings: +// - name: body type: text +// indexing_settings: +// commit_timeout_secs: 2 +// "# +// ); + +// // Create index +// sandbox +// .indexer_rest_client +// .indexes() +// .create(index_config, ConfigFormat::Yaml, false) +// .await +// .unwrap(); + +// sandbox.enable_ingest_v2(); + +// sandbox +// .indexer_rest_client +// .ingest( +// index_id, +// ingest_json!({"body": "wait"}), +// None, +// None, +// CommitType::WaitFor, +// ) +// .await +// .unwrap(); + +// sandbox.assert_hit_count(index_id, "body:wait", 1).await; + +// sandbox.shutdown().await.unwrap(); +// } + +#[tokio::test] +async fn test_commit_auto() { + initialize_tests(); + let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; + let index_id = "test_commit_auto"; + let index_config = format!( + r#" + version: 0.8 + index_id: {index_id} + doc_mapping: + field_mappings: + - name: body + type: text + indexing_settings: + commit_timeout_secs: 2 + "# + ); + sandbox .indexer_rest_client - .ingest( - index_id, - ingest_json!({"body": "wait"}), - None, - None, - CommitType::WaitFor, - ) + .indexes() + .create(index_config, ConfigFormat::Yaml, false) .await .unwrap(); - sandbox.assert_hit_count(index_id, "body:wait", 1).await; + sandbox.enable_ingest_v2(); - // Test auto commit sandbox .indexer_rest_client .ingest( @@ -388,13 +451,12 @@ async fn test_commit_modes() { sandbox.assert_hit_count(index_id, "body:auto", 0).await; sandbox - .wait_for_splits(index_id, Some(vec![SplitState::Published]), 3) + .wait_for_splits(index_id, Some(vec![SplitState::Published]), 1) .await .unwrap(); sandbox.assert_hit_count(index_id, "body:auto", 1).await; - // Clean up sandbox.shutdown().await.unwrap(); } From 911daa5b5b4e164b435ed3213a9323dddd596be2 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:18 +0200 Subject: [PATCH 05/25] Fix CommitTypeV2 serialization --- .../src/tests/ingest_tests.rs | 2 +- .../quickwit-serve/src/ingest_api/rest_handler.rs | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index 3412b35dbbb..248a4aaf113 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -353,7 +353,7 @@ async fn test_commit_force() { .await .unwrap(); - // commit_timeout_secs is set to a large value, so this will timeout if + // commit_timeout_secs is set to a large value, so this would timeout if // CommitType::Force is not working sandbox .wait_for_splits(index_id, Some(vec![SplitState::Published]), 1) diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index ee086902522..713b361bbeb 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -27,6 +27,7 @@ use quickwit_proto::ingest::router::{ IngestRequestV2, IngestResponseV2, IngestRouterService, IngestRouterServiceClient, IngestSubrequest, }; +use quickwit_proto::ingest::CommitTypeV2; use quickwit_proto::types::{DocUidGenerator, IndexId}; use serde::Deserialize; use warp::{Filter, Rejection}; @@ -56,6 +57,13 @@ struct IngestOptions { commit_type: CommitType, } +#[derive(Clone, Debug, Default, Deserialize, PartialEq)] +struct IngestV2Options { + #[serde(alias = "commit")] + #[serde(default)] + commit_type: CommitTypeV2, +} + pub(crate) fn ingest_api_handlers( ingest_router: IngestRouterServiceClient, ingest_service: IngestServiceClient, @@ -92,14 +100,14 @@ fn ingest_handler( fn ingest_v2_filter( config: IngestApiConfig, -) -> impl Filter + Clone { +) -> impl Filter + Clone { warp::path!(String / "ingest-v2") .and(warp::post()) .and(warp::body::content_length_limit( config.content_length_limit.as_u64(), )) .and(get_body_bytes()) - .and(serde_qs::warp::query::( + .and(serde_qs::warp::query::( serde_qs::Config::default(), )) } @@ -118,7 +126,7 @@ fn ingest_v2_handler( async fn ingest_v2( index_id: IndexId, body: Body, - ingest_options: IngestOptions, + ingest_options: IngestV2Options, ingest_router: IngestRouterServiceClient, ) -> Result { let mut doc_batch_builder = DocBatchV2Builder::default(); From e0fceb0929eeae984c079d6002a270398fe08212 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:18 +0200 Subject: [PATCH 06/25] Explicit 400 when using ingest V2 and wait_for --- quickwit/quickwit-ingest/src/error.rs | 4 + .../src/tests/ingest_tests.rs | 99 ++++++++++--------- .../protos/quickwit/ingest.proto | 2 +- .../src/codegen/quickwit/quickwit.ingest.rs | 6 +- .../src/elasticsearch_api/bulk_v2.rs | 9 +- .../model/bulk_query_params.rs | 2 +- .../src/ingest_api/rest_handler.rs | 6 ++ 7 files changed, 72 insertions(+), 56 deletions(-) diff --git a/quickwit/quickwit-ingest/src/error.rs b/quickwit/quickwit-ingest/src/error.rs index ab2c282db36..50391c8293a 100644 --- a/quickwit/quickwit-ingest/src/error.rs +++ b/quickwit/quickwit-ingest/src/error.rs @@ -32,6 +32,8 @@ use serde::{Deserialize, Serialize}; #[derive(Debug, Clone, thiserror::Error, Serialize, Deserialize)] pub enum IngestServiceError { + #[error("unimplemented: {0}")] + Unimplemented(String), #[error("data corruption: {0}")] Corruption(String), #[error("index `{index_id}` already exists")] @@ -141,6 +143,7 @@ impl From for IngestServiceError { impl ServiceError for IngestServiceError { fn error_code(&self) -> ServiceErrorCode { match self { + Self::Unimplemented(_) => ServiceErrorCode::BadRequest, Self::Corruption(err_msg) => { rate_limited_error!( limit_per_min = 6, @@ -196,6 +199,7 @@ impl From for IngestServiceError { impl From for tonic::Status { fn from(error: IngestServiceError) -> tonic::Status { let code = match &error { + IngestServiceError::Unimplemented(_) => tonic::Code::InvalidArgument, IngestServiceError::Corruption { .. } => tonic::Code::DataLoss, IngestServiceError::IndexAlreadyExists { .. } => tonic::Code::AlreadyExists, IngestServiceError::IndexNotFound { .. } => tonic::Code::NotFound, diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index 248a4aaf113..cbe2241cd16 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -219,7 +219,7 @@ async fn test_ingest_v2_index_not_found() { ingest_json!({"body": "doc1"}), None, None, - CommitType::WaitFor, + CommitType::Auto, ) .await .unwrap_err(); @@ -281,7 +281,7 @@ async fn test_ingest_v2_happy_path() { ingest_json!({"body": "doc1"}), None, None, - CommitType::WaitFor, + CommitType::Auto, ) .await; let Some(ingest_error) = ingest_res.err() else { @@ -365,49 +365,52 @@ async fn test_commit_force() { sandbox.shutdown().await.unwrap(); } -// #[tokio::test] -// async fn test_commit_wait_for() { -// initialize_tests(); -// let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; -// let index_id = "test_commit_wait_for"; -// let index_config = format!( -// r#" -// version: 0.8 -// index_id: {index_id} -// doc_mapping: -// field_mappings: -// - name: body type: text -// indexing_settings: -// commit_timeout_secs: 2 -// "# -// ); - -// // Create index -// sandbox -// .indexer_rest_client -// .indexes() -// .create(index_config, ConfigFormat::Yaml, false) -// .await -// .unwrap(); - -// sandbox.enable_ingest_v2(); - -// sandbox -// .indexer_rest_client -// .ingest( -// index_id, -// ingest_json!({"body": "wait"}), -// None, -// None, -// CommitType::WaitFor, -// ) -// .await -// .unwrap(); - -// sandbox.assert_hit_count(index_id, "body:wait", 1).await; - -// sandbox.shutdown().await.unwrap(); -// } +#[tokio::test] +async fn test_commit_wait_for() { + initialize_tests(); + let mut sandbox = ClusterSandboxBuilder::build_and_start_standalone().await; + let index_id = "test_commit_wait_for"; + let index_config = format!( + r#" + version: 0.8 + index_id: {index_id} + doc_mapping: + field_mappings: + - name: body + type: text + indexing_settings: + commit_timeout_secs: 2 + "# + ); + + // Create index + sandbox + .indexer_rest_client + .indexes() + .create(index_config, ConfigFormat::Yaml, false) + .await + .unwrap(); + + sandbox.enable_ingest_v2(); + + let ingest_error = sandbox + .indexer_rest_client + .ingest( + index_id, + ingest_json!({"body": "wait"}), + None, + None, + CommitType::WaitFor, + ) + .await + .unwrap_err(); + + // TODO https://github.com/quickwit-oss/quickwit/issues/5351 + assert_eq!(ingest_error.status_code(), Some(StatusCode::BAD_REQUEST)); + // sandbox.assert_hit_count(index_id, "body:wait", 1).await; + + sandbox.shutdown().await.unwrap(); +} #[tokio::test] async fn test_commit_auto() { @@ -507,7 +510,7 @@ async fn test_very_large_index_name() { &sandbox.indexer_rest_client, index_id, ingest_json!({"body": "not too long"}), - CommitType::WaitFor, + CommitType::Auto, ) .await .unwrap(); @@ -667,7 +670,7 @@ async fn test_shutdown_control_plane_early_shutdown() { &sandbox.indexer_rest_client, index_id, ingest_json!({"body": "one"}), - CommitType::WaitFor, + CommitType::Force, ) .await .unwrap(); @@ -726,7 +729,7 @@ async fn test_shutdown_separate_indexer() { &sandbox.indexer_rest_client, index_id, ingest_json!({"body": "one"}), - CommitType::WaitFor, + CommitType::Force, ) .await .unwrap(); diff --git a/quickwit/quickwit-proto/protos/quickwit/ingest.proto b/quickwit/quickwit-proto/protos/quickwit/ingest.proto index 404bbf7c660..0eb000b675f 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingest.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingest.proto @@ -49,7 +49,7 @@ message ShardPKey { enum CommitTypeV2 { COMMIT_TYPE_V2_UNSPECIFIED = 0; COMMIT_TYPE_V2_AUTO = 1; - COMMIT_TYPE_V2_WAIT = 2; + COMMIT_TYPE_V2_WAIT_FOR = 2; COMMIT_TYPE_V2_FORCE = 3; } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs index 9b071d20db4..0790c8a7416 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -122,7 +122,7 @@ pub struct ParseFailure { pub enum CommitTypeV2 { Unspecified = 0, Auto = 1, - Wait = 2, + WaitFor = 2, Force = 3, } impl CommitTypeV2 { @@ -134,7 +134,7 @@ impl CommitTypeV2 { match self { CommitTypeV2::Unspecified => "COMMIT_TYPE_V2_UNSPECIFIED", CommitTypeV2::Auto => "COMMIT_TYPE_V2_AUTO", - CommitTypeV2::Wait => "COMMIT_TYPE_V2_WAIT", + CommitTypeV2::WaitFor => "COMMIT_TYPE_V2_WAIT_FOR", CommitTypeV2::Force => "COMMIT_TYPE_V2_FORCE", } } @@ -143,7 +143,7 @@ impl CommitTypeV2 { match value { "COMMIT_TYPE_V2_UNSPECIFIED" => Some(Self::Unspecified), "COMMIT_TYPE_V2_AUTO" => Some(Self::Auto), - "COMMIT_TYPE_V2_WAIT" => Some(Self::Wait), + "COMMIT_TYPE_V2_WAIT_FOR" => Some(Self::WaitFor), "COMMIT_TYPE_V2_FORCE" => Some(Self::Force), _ => None, } diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs index e068483730a..c53079d8af5 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs @@ -30,7 +30,6 @@ use quickwit_proto::ingest::router::{ use quickwit_proto::ingest::CommitTypeV2; use quickwit_proto::types::{DocUid, IndexId}; use serde::{Deserialize, Serialize}; -use tracing::warn; use super::model::ElasticException; use crate::elasticsearch_api::model::{BulkAction, ElasticBulkOptions, ElasticsearchError}; @@ -140,8 +139,12 @@ pub(crate) async fn elastic_bulk_ingest_v2( } let commit_type: CommitTypeV2 = bulk_options.refresh.into(); - if commit_type != CommitTypeV2::Auto { - warn!("ingest API v2 does not support the `refresh` parameter (yet)"); + if commit_type == CommitTypeV2::WaitFor { + ElasticsearchError::new( + StatusCode::BAD_REQUEST, + "ingest API v2 does not support the `refresh=wait_for` parameter (yet)".to_string(), + Some(ElasticException::IllegalArgument), + ); } let ingest_request_opt = ingest_request_builder.build(INGEST_V2_SOURCE_ID, commit_type); diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/model/bulk_query_params.rs b/quickwit/quickwit-serve/src/elasticsearch_api/model/bulk_query_params.rs index a1d982b4e25..e9b415c8248 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/model/bulk_query_params.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/model/bulk_query_params.rs @@ -68,7 +68,7 @@ impl From for CommitTypeV2 { match val { ElasticRefresh::False => Self::Auto, ElasticRefresh::True => Self::Force, - ElasticRefresh::WaitFor => Self::Wait, + ElasticRefresh::WaitFor => Self::WaitFor, } } } diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index 713b361bbeb..37c72a1f5b3 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -129,6 +129,12 @@ async fn ingest_v2( ingest_options: IngestV2Options, ingest_router: IngestRouterServiceClient, ) -> Result { + if ingest_options.commit_type == CommitTypeV2::WaitFor { + return Err(IngestServiceError::Unimplemented( + "ingest API v2 does not support the `refrcommitesh=wait_for` parameter (yet)" + .to_string(), + )); + } let mut doc_batch_builder = DocBatchV2Builder::default(); let mut doc_uid_generator = DocUidGenerator::default(); From 07cd7eb3b3e4f251e25ff134d8fb6e09b0130fb6 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:19 +0200 Subject: [PATCH 07/25] Fix wait_for commit on ingest V2 --- quickwit/quickwit-ingest/src/error.rs | 4 - .../quickwit-ingest/src/ingest_v2/router.rs | 38 ++- .../src/ingest_v2/workbench.rs | 220 ++++++++++++++++-- .../src/tests/ingest_tests.rs | 8 +- .../src/elasticsearch_api/bulk_v2.rs | 7 - .../src/ingest_api/rest_handler.rs | 6 - quickwit/quickwit-serve/src/lib.rs | 3 +- 7 files changed, 240 insertions(+), 46 deletions(-) diff --git a/quickwit/quickwit-ingest/src/error.rs b/quickwit/quickwit-ingest/src/error.rs index 50391c8293a..ab2c282db36 100644 --- a/quickwit/quickwit-ingest/src/error.rs +++ b/quickwit/quickwit-ingest/src/error.rs @@ -32,8 +32,6 @@ use serde::{Deserialize, Serialize}; #[derive(Debug, Clone, thiserror::Error, Serialize, Deserialize)] pub enum IngestServiceError { - #[error("unimplemented: {0}")] - Unimplemented(String), #[error("data corruption: {0}")] Corruption(String), #[error("index `{index_id}` already exists")] @@ -143,7 +141,6 @@ impl From for IngestServiceError { impl ServiceError for IngestServiceError { fn error_code(&self) -> ServiceErrorCode { match self { - Self::Unimplemented(_) => ServiceErrorCode::BadRequest, Self::Corruption(err_msg) => { rate_limited_error!( limit_per_min = 6, @@ -199,7 +196,6 @@ impl From for IngestServiceError { impl From for tonic::Status { fn from(error: IngestServiceError) -> tonic::Status { let code = match &error { - IngestServiceError::Unimplemented(_) => tonic::Code::InvalidArgument, IngestServiceError::Corruption { .. } => tonic::Code::DataLoss, IngestServiceError::IndexAlreadyExists { .. } => tonic::Code::AlreadyExists, IngestServiceError::IndexNotFound { .. } => tonic::Code::NotFound, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 4f46ed5ea92..7a94d46f0e2 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -101,6 +101,7 @@ pub struct IngestRouter { replication_factor: usize, // Limits the number of ingest requests in-flight to some capacity in bytes. ingest_semaphore: Arc, + event_broker: EventBroker, } struct RouterState { @@ -125,6 +126,7 @@ impl IngestRouter { control_plane: ControlPlaneServiceClient, ingester_pool: IngesterPool, replication_factor: usize, + event_broker: EventBroker, ) -> Self { let state = Arc::new(Mutex::new(RouterState { debouncer: GetOrCreateOpenShardsRequestDebouncer::default(), @@ -143,15 +145,16 @@ impl IngestRouter { state, replication_factor, ingest_semaphore, + event_broker, } } - pub fn subscribe(&self, event_broker: &EventBroker) { + pub fn subscribe(&self) { let weak_router_state = WeakRouterState(Arc::downgrade(&self.state)); - event_broker + self.event_broker .subscribe::(weak_router_state.clone()) .forever(); - event_broker + self.event_broker .subscribe::(weak_router_state) .forever(); } @@ -454,12 +457,20 @@ impl IngestRouter { max_num_attempts: usize, ) -> IngestResponseV2 { let commit_type = ingest_request.commit_type(); - let mut workbench = IngestWorkbench::new(ingest_request.subrequests, max_num_attempts); + let mut workbench = if commit_type == CommitTypeV2::WaitFor { + IngestWorkbench::new_with_publish_tracking( + ingest_request.subrequests, + max_num_attempts, + self.event_broker.clone(), + ) + } else { + IngestWorkbench::new(ingest_request.subrequests, max_num_attempts) + }; while !workbench.is_complete() { workbench.new_attempt(); self.batch_persist(&mut workbench, commit_type).await; } - workbench.into_ingest_result() + workbench.into_ingest_result().await } async fn ingest_timeout( @@ -712,6 +723,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let mut workbench = IngestWorkbench::default(); let (get_or_create_open_shard_request_opt, rendezvous) = router @@ -948,6 +960,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let ingest_subrequests = vec![ IngestSubrequest { @@ -1062,6 +1075,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1120,6 +1134,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1149,6 +1164,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1200,6 +1216,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1251,6 +1268,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); let mut state_guard = router.state.lock().await; @@ -1337,6 +1355,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let ingest_subrequests = vec![ IngestSubrequest { @@ -1416,6 +1435,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); let index_uid2: IndexUid = IndexUid::for_test("test-index-1", 0); @@ -1653,6 +1673,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let mut state_guard = router.state.lock().await; let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); @@ -1757,14 +1778,15 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); let replication_factor = 1; + let event_broker = EventBroker::default(); let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), replication_factor, + event_broker.clone(), ); - let event_broker = EventBroker::default(); - router.subscribe(&event_broker); + router.subscribe(); let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); let mut state_guard = router.state.lock().await; @@ -1854,6 +1876,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let index_uid_0: IndexUid = IndexUid::for_test("test-index-0", 0); let index_uid_1: IndexUid = IndexUid::for_test("test-index-1", 0); @@ -1903,6 +1926,7 @@ mod tests { control_plane, ingester_pool.clone(), replication_factor, + EventBroker::default(), ); let mut state_guard = router.state.lock().await; let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index a8bc0700270..1cdcf118976 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -17,22 +17,102 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::{BTreeMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; +use std::sync::{Arc, Mutex}; +use quickwit_common::pubsub::{EventBroker, EventSubscriptionHandle}; use quickwit_common::rate_limited_error; use quickwit_proto::control_plane::{ GetOrCreateOpenShardsFailure, GetOrCreateOpenShardsFailureReason, }; +use quickwit_proto::indexing::ShardPositionsUpdate; use quickwit_proto::ingest::ingester::{PersistFailure, PersistFailureReason, PersistSuccess}; use quickwit_proto::ingest::router::{ IngestFailure, IngestFailureReason, IngestResponseV2, IngestSubrequest, IngestSuccess, }; use quickwit_proto::ingest::{IngestV2Error, RateLimitingCause}; -use quickwit_proto::types::{NodeId, ShardId, SubrequestId}; +use quickwit_proto::types::{NodeId, Position, ShardId, SubrequestId}; +use tokio::sync::Notify; use tracing::warn; use super::router::PersistRequestSummary; +#[derive(Default)] +struct PublishState { + awaiting_publish: HashMap, + already_published: HashMap, +} + +/// A helper for tracking the progress of the publish events when running in +/// `wait_for` commit mode. +/// +/// Registers a set of shard positions and listens to [`ShardPositionsUpdate`] +/// events to assert when all the persisted events have been published. To make +/// sure that no events are missed: +/// - the tracker should be created before the persist requests are sent +/// - the `shard_persisted` method should for all successful persist subrequests +struct PublishTracker { + state: Arc>, + publish_complete: Arc, + _publish_listen_handle: EventSubscriptionHandle, +} + +impl PublishTracker { + fn new(event_tracker: EventBroker) -> Self { + let state = Arc::new(Mutex::new(PublishState::default())); + let state_clone = state.clone(); + let publish_complete = Arc::new(Notify::new()); + let publish_complete_notifier = publish_complete.clone(); + let _publish_listen_handle = + event_tracker.subscribe(move |update: ShardPositionsUpdate| { + let mut state_handle = state_clone.lock().unwrap(); + for (updated_shard_id, updated_position) in &update.updated_shard_positions { + if let Some(shard_position) = + state_handle.awaiting_publish.get(updated_shard_id) + { + if updated_position >= shard_position { + state_handle.awaiting_publish.remove(updated_shard_id); + if state_handle.awaiting_publish.is_empty() { + publish_complete_notifier.notify_one(); + } + } + } else { + // Save this position update in case the publish update + // event arrived before the shard persist response. We + // might build a state that tracks irrelevant shards for + // the duration of the query but that should be fine. + state_handle + .already_published + .insert(updated_shard_id.clone(), updated_position.clone()); + } + } + }); + Self { + state, + _publish_listen_handle, + publish_complete, + } + } + + fn shard_persisted(&self, shard_id: ShardId, new_position: Position) { + let mut state_handle = self.state.lock().unwrap(); + match state_handle.already_published.get(&shard_id) { + Some(already_published_position) if new_position <= *already_published_position => { + // already published, no need to track this shard's position updates + } + _ => { + state_handle + .awaiting_publish + .insert(shard_id.clone(), new_position.clone()); + } + } + } + + async fn wait_publish_complete(self) { + self.publish_complete.notified().await; + } +} + /// A helper struct for managing the state of the subrequests of an ingest request during multiple /// persist attempts. #[derive(Default)] @@ -44,13 +124,14 @@ pub(super) struct IngestWorkbench { /// subrequest. pub num_attempts: usize, pub max_num_attempts: usize, - // List of leaders that have been marked as temporarily unavailable. - // These leaders have encountered a transport error during an attempt and will be treated as if - // they were out of the pool for subsequent attempts. - // - // (The point here is to make sure we do not wait for the failure detection to kick the node - // out of the ingest node.) + /// List of leaders that have been marked as temporarily unavailable. + /// These leaders have encountered a transport error during an attempt and will be treated as + /// if they were out of the pool for subsequent attempts. + /// + /// (The point here is to make sure we do not wait for the failure detection to kick the node + /// out of the ingest node.) pub unavailable_leaders: HashSet, + publish_tracker: Option, } /// Returns an iterator of pending of subrequests, sorted by sub request id. @@ -67,7 +148,11 @@ pub(super) fn pending_subrequests( } impl IngestWorkbench { - pub fn new(ingest_subrequests: Vec, max_num_attempts: usize) -> Self { + fn new_inner( + ingest_subrequests: Vec, + max_num_attempts: usize, + publish_tracker: Option, + ) -> Self { let subworkbenches: BTreeMap = ingest_subrequests .into_iter() .map(|subrequest| { @@ -81,10 +166,27 @@ impl IngestWorkbench { Self { subworkbenches, max_num_attempts, + publish_tracker, ..Default::default() } } + pub fn new(ingest_subrequests: Vec, max_num_attempts: usize) -> Self { + Self::new_inner(ingest_subrequests, max_num_attempts, None) + } + + pub fn new_with_publish_tracking( + ingest_subrequests: Vec, + max_num_attempts: usize, + event_broker: EventBroker, + ) -> Self { + Self::new_inner( + ingest_subrequests, + max_num_attempts, + Some(PublishTracker::new(event_broker)), + ) + } + pub fn new_attempt(&mut self) { self.num_attempts += 1; } @@ -138,6 +240,12 @@ impl IngestWorkbench { ); return; }; + if let Some(publish_tracker) = &mut self.publish_tracker { + if let Some(position) = &persist_success.replication_position_inclusive { + publish_tracker + .shard_persisted(persist_success.shard_id().clone(), position.clone()); + } + } self.num_successes += 1; subworkbench.num_attempts += 1; subworkbench.persist_success_opt = Some(persist_success); @@ -223,7 +331,7 @@ impl IngestWorkbench { ); } - pub fn into_ingest_result(self) -> IngestResponseV2 { + pub async fn into_ingest_result(self) -> IngestResponseV2 { let num_subworkbenches = self.subworkbenches.len(); let mut successes = Vec::with_capacity(self.num_successes); let mut failures = Vec::with_capacity(num_subworkbenches - self.num_successes); @@ -255,6 +363,10 @@ impl IngestWorkbench { let num_failures = failures.len(); assert_eq!(num_successes + num_failures, num_subworkbenches); + if let Some(publish_tracker) = self.publish_tracker { + publish_tracker.wait_publish_complete().await; + } + // For tests, we sort the successes and failures by subrequest_id #[cfg(test)] { @@ -358,11 +470,87 @@ impl IngestSubworkbench { #[cfg(test)] mod tests { + use std::time::Duration; + use quickwit_proto::ingest::ingester::PersistFailureReason; - use quickwit_proto::types::ShardId; + use quickwit_proto::types::{IndexUid, ShardId, SourceUid}; use super::*; + #[tokio::test] + async fn test_publish_tracker() { + let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); + let shard_id_3 = ShardId::from("test-shard-3"); + let shard_id_4 = ShardId::from("test-shard-3"); + let position = Position::offset(42usize); + let greater_position = Position::offset(666usize); + tracker.shard_persisted(shard_id_1.clone(), position.clone()); + tracker.shard_persisted(shard_id_2.clone(), position.clone()); + tracker.shard_persisted(shard_id_3.clone(), position.clone()); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![ + (shard_id_1.clone(), position.clone()), + (shard_id_2.clone(), greater_position), + ] + .into_iter() + .collect(), + }); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![ + (shard_id_3.clone(), position.clone()), + (shard_id_4.clone(), position.clone()), + ] + .into_iter() + .collect(), + }); + + // persist response received after the publish event + tracker.shard_persisted(shard_id_4.clone(), position.clone()); + + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap(); + } + + #[tokio::test] + async fn test_publish_tracker_waits() { + let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + let shard_id_1 = ShardId::from("test-shard-1"); + let position = Position::offset(42usize); + tracker.shard_persisted(shard_id_1.clone(), position.clone()); + tracker.shard_persisted(ShardId::from("test-shard-2"), position.clone()); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] + .into_iter() + .collect(), + }); + + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap_err(); + } + #[test] fn test_ingest_subworkbench() { let subrequest = IngestSubrequest { @@ -680,10 +868,10 @@ mod tests { assert_eq!(subworkbench.num_attempts, 1); } - #[test] - fn test_ingest_workbench_into_ingest_result() { + #[tokio::test] + async fn test_ingest_workbench_into_ingest_result() { let workbench = IngestWorkbench::new(Vec::new(), 0); - let response = workbench.into_ingest_result(); + let response = workbench.into_ingest_result().await; assert!(response.successes.is_empty()); assert!(response.failures.is_empty()); @@ -706,7 +894,7 @@ mod tests { workbench.record_no_shards_available(1); - let response = workbench.into_ingest_result(); + let response = workbench.into_ingest_result().await; assert_eq!(response.successes.len(), 1); assert_eq!(response.successes[0].subrequest_id, 0); @@ -725,7 +913,7 @@ mod tests { let failure = SubworkbenchFailure::Persist(PersistFailureReason::Timeout); workbench.record_failure(0, failure); - let ingest_response = workbench.into_ingest_result(); + let ingest_response = workbench.into_ingest_result().await; assert_eq!(ingest_response.successes.len(), 0); assert_eq!( ingest_response.failures[0].reason(), diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index cbe2241cd16..40cfa8e041b 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -393,7 +393,7 @@ async fn test_commit_wait_for() { sandbox.enable_ingest_v2(); - let ingest_error = sandbox + sandbox .indexer_rest_client .ingest( index_id, @@ -403,11 +403,9 @@ async fn test_commit_wait_for() { CommitType::WaitFor, ) .await - .unwrap_err(); + .unwrap(); - // TODO https://github.com/quickwit-oss/quickwit/issues/5351 - assert_eq!(ingest_error.status_code(), Some(StatusCode::BAD_REQUEST)); - // sandbox.assert_hit_count(index_id, "body:wait", 1).await; + sandbox.assert_hit_count(index_id, "body:wait", 1).await; sandbox.shutdown().await.unwrap(); } diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs index c53079d8af5..a90cc9f5edf 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs @@ -139,13 +139,6 @@ pub(crate) async fn elastic_bulk_ingest_v2( } let commit_type: CommitTypeV2 = bulk_options.refresh.into(); - if commit_type == CommitTypeV2::WaitFor { - ElasticsearchError::new( - StatusCode::BAD_REQUEST, - "ingest API v2 does not support the `refresh=wait_for` parameter (yet)".to_string(), - Some(ElasticException::IllegalArgument), - ); - } let ingest_request_opt = ingest_request_builder.build(INGEST_V2_SOURCE_ID, commit_type); let Some(ingest_request) = ingest_request_opt else { diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index 37c72a1f5b3..713b361bbeb 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -129,12 +129,6 @@ async fn ingest_v2( ingest_options: IngestV2Options, ingest_router: IngestRouterServiceClient, ) -> Result { - if ingest_options.commit_type == CommitTypeV2::WaitFor { - return Err(IngestServiceError::Unimplemented( - "ingest API v2 does not support the `refrcommitesh=wait_for` parameter (yet)" - .to_string(), - )); - } let mut doc_batch_builder = DocBatchV2Builder::default(); let mut doc_uid_generator = DocUidGenerator::default(); diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 4de79fe5d1b..9b816333528 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -864,8 +864,9 @@ async fn setup_ingest_v2( control_plane.clone(), ingester_pool.clone(), replication_factor, + event_broker.clone(), ); - ingest_router.subscribe(event_broker); + ingest_router.subscribe(); let ingest_router_service = IngestRouterServiceClient::tower() .stack_layer(INGEST_GRPC_SERVER_METRICS_LAYER.clone()) From d48819950f430ea6500739ed1e2e37ee3a17a29b Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:19 +0200 Subject: [PATCH 08/25] Also wait when the commit type is force --- .../quickwit-ingest/src/ingest_v2/router.rs | 2 +- .../src/tests/ingest_tests.rs | 26 +++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 7a94d46f0e2..f403b840479 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -457,7 +457,7 @@ impl IngestRouter { max_num_attempts: usize, ) -> IngestResponseV2 { let commit_type = ingest_request.commit_type(); - let mut workbench = if commit_type == CommitTypeV2::WaitFor { + let mut workbench = if matches!(commit_type, CommitTypeV2::Force | CommitTypeV2::WaitFor) { IngestWorkbench::new_with_publish_tracking( ingest_request.subrequests, max_num_attempts, diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index 40cfa8e041b..4ccea554511 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -330,7 +330,7 @@ async fn test_commit_force() { - name: body type: text indexing_settings: - commit_timeout_secs: 20 + commit_timeout_secs: 60 "# ); @@ -344,22 +344,21 @@ async fn test_commit_force() { sandbox.enable_ingest_v2(); - ingest_with_retry( - &sandbox.indexer_rest_client, - index_id, - ingest_json!({"body": "force"}), - CommitType::Force, + // commit_timeout_secs is set to a large value, so this would timeout if + // the commit isn't forced + tokio::time::timeout( + Duration::from_secs(20), + ingest_with_retry( + &sandbox.indexer_rest_client, + index_id, + ingest_json!({"body": "force"}), + CommitType::Force, + ), ) .await + .unwrap() .unwrap(); - // commit_timeout_secs is set to a large value, so this would timeout if - // CommitType::Force is not working - sandbox - .wait_for_splits(index_id, Some(vec![SplitState::Published]), 1) - .await - .unwrap(); - sandbox.assert_hit_count(index_id, "body:force", 1).await; sandbox.shutdown().await.unwrap(); @@ -405,6 +404,7 @@ async fn test_commit_wait_for() { .await .unwrap(); + // This test is not powerful enough to ensure that `wait_for` does not force the commit sandbox.assert_hit_count(index_id, "body:wait", 1).await; sandbox.shutdown().await.unwrap(); From 38fa01bac025d1011f41a7f2c3af2c5bd3338f68 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:19 +0200 Subject: [PATCH 09/25] Assert wait_for doesn't force --- .../src/tests/ingest_tests.rs | 48 ++++++++++++++++--- 1 file changed, 42 insertions(+), 6 deletions(-) diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index 4ccea554511..77a8bb800b5 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -19,12 +19,14 @@ use std::time::Duration; +use futures_util::FutureExt; use hyper::StatusCode; use quickwit_config::service::QuickwitService; use quickwit_config::ConfigFormat; use quickwit_metastore::SplitState; use quickwit_rest_client::error::{ApiError, Error}; use quickwit_rest_client::rest_client::CommitType; +use quickwit_serve::ListSplitsQueryParams; use serde_json::json; use crate::ingest_json; @@ -378,7 +380,7 @@ async fn test_commit_wait_for() { - name: body type: text indexing_settings: - commit_timeout_secs: 2 + commit_timeout_secs: 3 "# ); @@ -392,20 +394,54 @@ async fn test_commit_wait_for() { sandbox.enable_ingest_v2(); - sandbox + // run 2 ingest requests at the same time on the same index + // wait_for shouldn't force the commit so expect only 1 published split + + let ingest_1_fut = sandbox .indexer_rest_client .ingest( index_id, - ingest_json!({"body": "wait"}), + ingest_json!({"body": "wait for"}), None, None, CommitType::WaitFor, ) + .then(|res| async { + let resp = res.unwrap(); + sandbox.assert_hit_count(index_id, "body:for", 1).await; + resp + }); + + let ingest_2_fut = sandbox + .indexer_rest_client + .ingest( + index_id, + ingest_json!({"body": "wait again"}), + None, + None, + CommitType::WaitFor, + ) + .then(|res| async { + let resp = res.unwrap(); + sandbox.assert_hit_count(index_id, "body:again", 1).await; + resp + }); + + tokio::join!(ingest_1_fut, ingest_2_fut); + + sandbox.assert_hit_count(index_id, "body:wait", 2).await; + + let splits_query_params = ListSplitsQueryParams { + split_states: Some(vec![SplitState::Published]), + ..Default::default() + }; + let published_splits = sandbox + .indexer_rest_client + .splits(index_id) + .list(splits_query_params) .await .unwrap(); - - // This test is not powerful enough to ensure that `wait_for` does not force the commit - sandbox.assert_hit_count(index_id, "body:wait", 1).await; + assert_eq!(published_splits.len(), 1); sandbox.shutdown().await.unwrap(); } From c4d7ecbec3ada5e27e830e91971b3cf8f7f709a7 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:19 +0200 Subject: [PATCH 10/25] Test and fix edge case with empty workbench --- .../src/ingest_v2/workbench.rs | 129 ++++++++++++++++-- 1 file changed, 117 insertions(+), 12 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 1cdcf118976..56365ecd9b1 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -109,6 +109,12 @@ impl PublishTracker { } async fn wait_publish_complete(self) { + // correctness: + // - `state.awaiting_publish` can only grow before this (`self` is consumed) + // - `publish_complete.notify()` is called as soon as the last shard is published + if self.state.lock().unwrap().awaiting_publish.is_empty() { + return; + } self.publish_complete.notified().await; } } @@ -191,8 +197,8 @@ impl IngestWorkbench { self.num_attempts += 1; } - /// Returns true if all subrequests were successful or if the number of - /// attempts has been exhausted. + /// Returns true if all subrequests were successfully persisted or if the + /// number of attempts has been exhausted. pub fn is_complete(&self) -> bool { self.num_successes >= self.subworkbenches.len() || self.num_attempts >= self.max_num_attempts @@ -486,11 +492,10 @@ mod tests { let shard_id_2 = ShardId::from("test-shard-2"); let shard_id_3 = ShardId::from("test-shard-3"); let shard_id_4 = ShardId::from("test-shard-3"); - let position = Position::offset(42usize); - let greater_position = Position::offset(666usize); - tracker.shard_persisted(shard_id_1.clone(), position.clone()); - tracker.shard_persisted(shard_id_2.clone(), position.clone()); - tracker.shard_persisted(shard_id_3.clone(), position.clone()); + + tracker.shard_persisted(shard_id_1.clone(), Position::offset(42usize)); + tracker.shard_persisted(shard_id_2.clone(), Position::offset(42usize)); + tracker.shard_persisted(shard_id_3.clone(), Position::offset(42usize)); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { @@ -498,8 +503,8 @@ mod tests { source_id: "test-source".to_string(), }, updated_shard_positions: vec![ - (shard_id_1.clone(), position.clone()), - (shard_id_2.clone(), greater_position), + (shard_id_1.clone(), Position::offset(42usize)), + (shard_id_2.clone(), Position::offset(666usize)), ] .into_iter() .collect(), @@ -511,15 +516,15 @@ mod tests { source_id: "test-source".to_string(), }, updated_shard_positions: vec![ - (shard_id_3.clone(), position.clone()), - (shard_id_4.clone(), position.clone()), + (shard_id_3.clone(), Position::eof(42usize)), + (shard_id_4.clone(), Position::offset(42usize)), ] .into_iter() .collect(), }); // persist response received after the publish event - tracker.shard_persisted(shard_id_4.clone(), position.clone()); + tracker.shard_persisted(shard_id_4.clone(), Position::offset(42usize)); tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) .await @@ -673,6 +678,106 @@ mod tests { assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 0); } + #[tokio::test] + async fn test_ingest_workbench_with_publish_tracking() { + let event_broker = EventBroker::default(); + let workbench = + IngestWorkbench::new_with_publish_tracking(Vec::new(), 1, event_broker.clone()); + assert!(workbench.is_complete()); + assert_eq!( + workbench.into_ingest_result().await, + IngestResponseV2::default() + ); + let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); + let ingest_subrequests = vec![ + IngestSubrequest { + subrequest_id: 0, + ..Default::default() + }, + IngestSubrequest { + subrequest_id: 1, + ..Default::default() + }, + ]; + let mut workbench = + IngestWorkbench::new_with_publish_tracking(ingest_subrequests, 1, event_broker.clone()); + assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 2); + assert!(!workbench.is_complete()); + + let persist_success = PersistSuccess { + subrequest_id: 0, + shard_id: Some(shard_id_1.clone()), + replication_position_inclusive: Some(Position::offset(42usize)), + ..Default::default() + }; + workbench.record_persist_success(persist_success); + + assert_eq!(workbench.num_successes, 1); + assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 1); + assert_eq!( + pending_subrequests(&workbench.subworkbenches) + .next() + .unwrap() + .subrequest_id, + 1 + ); + + let subworkbench = workbench.subworkbenches.get(&0).unwrap(); + assert_eq!(subworkbench.num_attempts, 1); + assert!(!subworkbench.is_pending()); + + let persist_failure = PersistFailure { + subrequest_id: 1, + shard_id: Some(shard_id_2.clone()), + ..Default::default() + }; + workbench.record_persist_failure(&persist_failure); + + assert_eq!(workbench.num_successes, 1); + assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 1); + assert_eq!( + pending_subrequests(&workbench.subworkbenches) + .next() + .unwrap() + .subrequest_id, + 1 + ); + + let subworkbench = workbench.subworkbenches.get(&1).unwrap(); + assert_eq!(subworkbench.num_attempts, 1); + assert!(subworkbench.last_failure_opt.is_some()); + + let persist_success = PersistSuccess { + subrequest_id: 1, + shard_id: Some(shard_id_2.clone()), + replication_position_inclusive: Some(Position::offset(66usize)), + ..Default::default() + }; + workbench.record_persist_success(persist_success); + + assert!(workbench.is_complete()); + assert_eq!(workbench.num_successes, 2); + assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 0); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: IndexUid::for_test("test-index", 0), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![ + (shard_id_1, Position::offset(42usize)), + (shard_id_2, Position::offset(66usize)), + ] + .into_iter() + .collect(), + }); + + let ingest_response = workbench.into_ingest_result().await; + assert_eq!(ingest_response.successes.len(), 2); + assert_eq!(ingest_response.failures.len(), 0); + } + #[test] fn test_ingest_workbench_record_get_or_create_open_shards_failure() { let ingest_subrequests = vec![IngestSubrequest { From c0c92d35e39ee44a19ae0fb9dc605eafed98bb6f Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:20 +0200 Subject: [PATCH 11/25] Add workbench test for publish wait --- .../src/ingest_v2/workbench.rs | 91 ++++++++++++------- 1 file changed, 60 insertions(+), 31 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 56365ecd9b1..388a990086b 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -679,15 +679,19 @@ mod tests { } #[tokio::test] - async fn test_ingest_workbench_with_publish_tracking() { - let event_broker = EventBroker::default(); + async fn test_workbench_publish_tracking_empty() { let workbench = - IngestWorkbench::new_with_publish_tracking(Vec::new(), 1, event_broker.clone()); + IngestWorkbench::new_with_publish_tracking(Vec::new(), 1, EventBroker::default()); assert!(workbench.is_complete()); assert_eq!( workbench.into_ingest_result().await, IngestResponseV2::default() ); + } + + #[tokio::test] + async fn test_workbench_publish_tracking_happy_path() { + let event_broker = EventBroker::default(); let shard_id_1 = ShardId::from("test-shard-1"); let shard_id_2 = ShardId::from("test-shard-2"); let ingest_subrequests = vec![ @@ -713,20 +717,6 @@ mod tests { }; workbench.record_persist_success(persist_success); - assert_eq!(workbench.num_successes, 1); - assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 1); - assert_eq!( - pending_subrequests(&workbench.subworkbenches) - .next() - .unwrap() - .subrequest_id, - 1 - ); - - let subworkbench = workbench.subworkbenches.get(&0).unwrap(); - assert_eq!(subworkbench.num_attempts, 1); - assert!(!subworkbench.is_pending()); - let persist_failure = PersistFailure { subrequest_id: 1, shard_id: Some(shard_id_2.clone()), @@ -734,20 +724,6 @@ mod tests { }; workbench.record_persist_failure(&persist_failure); - assert_eq!(workbench.num_successes, 1); - assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 1); - assert_eq!( - pending_subrequests(&workbench.subworkbenches) - .next() - .unwrap() - .subrequest_id, - 1 - ); - - let subworkbench = workbench.subworkbenches.get(&1).unwrap(); - assert_eq!(subworkbench.num_attempts, 1); - assert!(subworkbench.last_failure_opt.is_some()); - let persist_success = PersistSuccess { subrequest_id: 1, shard_id: Some(shard_id_2.clone()), @@ -778,6 +754,59 @@ mod tests { assert_eq!(ingest_response.failures.len(), 0); } + #[tokio::test] + async fn test_workbench_publish_tracking_waits() { + let event_broker = EventBroker::default(); + let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); + let ingest_subrequests = vec![ + IngestSubrequest { + subrequest_id: 0, + ..Default::default() + }, + IngestSubrequest { + subrequest_id: 1, + ..Default::default() + }, + ]; + let mut workbench = + IngestWorkbench::new_with_publish_tracking(ingest_subrequests, 1, event_broker.clone()); + + let persist_success = PersistSuccess { + subrequest_id: 0, + shard_id: Some(shard_id_1.clone()), + replication_position_inclusive: Some(Position::offset(42usize)), + ..Default::default() + }; + workbench.record_persist_success(persist_success); + + let persist_success = PersistSuccess { + subrequest_id: 1, + shard_id: Some(shard_id_2.clone()), + replication_position_inclusive: Some(Position::offset(66usize)), + ..Default::default() + }; + workbench.record_persist_success(persist_success); + + assert!(workbench.is_complete()); + assert_eq!(workbench.num_successes, 2); + assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 0); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: IndexUid::for_test("test-index", 0), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![(shard_id_2, Position::offset(66usize))] + .into_iter() + .collect(), + }); + // still waits for shard 1 to be published + tokio::time::timeout(Duration::from_millis(200), workbench.into_ingest_result()) + .await + .unwrap_err(); + } + #[test] fn test_ingest_workbench_record_get_or_create_open_shards_failure() { let ingest_subrequests = vec![IngestSubrequest { From f783cc63253bc055a61536174de5c80d75233f1a Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:20 +0200 Subject: [PATCH 12/25] Add ES tests --- .../src/elasticsearch_api/bulk_v2.rs | 36 +++++++++++++++++++ .../es_compatibility/_setup.quickwit.yaml | 1 - 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs index a90cc9f5edf..1220fcc415c 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs @@ -777,4 +777,40 @@ mod tests { assert_eq!(error.exception, ElasticException::IndexNotFound); assert_eq!(error.reason, "no such index [test-index-bar]"); } + + #[tokio::test] + async fn test_refresh_param() { + let mut mock_ingest_router = MockIngestRouterService::new(); + mock_ingest_router + .expect_ingest() + .once() + .returning(|ingest_request| { + assert_eq!(ingest_request.commit_type(), CommitTypeV2::WaitFor); + Ok(IngestResponseV2 { + successes: vec![IngestSuccess { + subrequest_id: 0, + index_uid: Some(IndexUid::for_test("my-index-1", 0)), + 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(), + }], + failures: Vec::new(), + }) + }); + let ingest_router = IngestRouterServiceClient::from_mock(mock_ingest_router); + let handler = es_compat_bulk_handler_v2(ingest_router); + + let payload = r#" + {"create": {"_index": "my-index-1", "_id" : "1"}} + {"ts": 1, "message": "my-message-1"} + "#; + warp::test::request() + .path("/_elastic/_bulk?refresh=wait_for") + .method("POST") + .body(payload) + .reply(&handler) + .await; + } } diff --git a/quickwit/rest-api-tests/scenarii/es_compatibility/_setup.quickwit.yaml b/quickwit/rest-api-tests/scenarii/es_compatibility/_setup.quickwit.yaml index ec6e9f81a3d..b6d2df1053f 100644 --- a/quickwit/rest-api-tests/scenarii/es_compatibility/_setup.quickwit.yaml +++ b/quickwit/rest-api-tests/scenarii/es_compatibility/_setup.quickwit.yaml @@ -66,4 +66,3 @@ params: refresh: "true" headers: {"Content-Type": "application/json", "content-encoding": "gzip"} body_from_file: gharchive-bulk.json.gz -sleep_after: 3 From cb34222dc275d87738b08b18663fbb0fbb454ac5 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:20 +0200 Subject: [PATCH 13/25] Fix clippy --- .../quickwit-integration-tests/src/tests/ingest_tests.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs index 77a8bb800b5..915a24dd7a4 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_tests.rs @@ -407,9 +407,8 @@ async fn test_commit_wait_for() { CommitType::WaitFor, ) .then(|res| async { - let resp = res.unwrap(); + res.unwrap(); sandbox.assert_hit_count(index_id, "body:for", 1).await; - resp }); let ingest_2_fut = sandbox @@ -422,9 +421,8 @@ async fn test_commit_wait_for() { CommitType::WaitFor, ) .then(|res| async { - let resp = res.unwrap(); + res.unwrap(); sandbox.assert_hit_count(index_id, "body:again", 1).await; - resp }); tokio::join!(ingest_1_fut, ingest_2_fut); From 757257bca1184c73dbac478d645dae1d86aa53d1 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 29 Aug 2024 15:01:20 +0200 Subject: [PATCH 14/25] Disable timeout on waiting commit types --- quickwit/quickwit-ingest/src/ingest_v2/router.rs | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index f403b840479..b999e6e57fb 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -606,9 +606,14 @@ impl IngestRouterService for IngestRouter { .try_acquire_many_owned(request_size_bytes as u32) .map_err(|_| IngestV2Error::TooManyRequests(RateLimitingCause::RouterLoadShedding))?; - let ingest_res = self - .ingest_timeout(ingest_request, ingest_request_timeout()) - .await; + let ingest_res = if ingest_request.commit_type() == CommitTypeV2::Auto { + self.ingest_timeout(ingest_request, ingest_request_timeout()) + .await + } else { + Ok(self + .retry_batch_persist(ingest_request, MAX_PERSIST_ATTEMPTS) + .await) + }; update_ingest_metrics(&ingest_res, num_subrequests); From 08504ccfe4776ae5c6f2abbff0b06f80711ad2a2 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 30 Aug 2024 11:08:54 +0200 Subject: [PATCH 15/25] Small naming and trait bound adjustments --- quickwit/quickwit-common/src/pubsub.rs | 2 +- .../quickwit-ingest/src/ingest_v2/workbench.rs | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/quickwit/quickwit-common/src/pubsub.rs b/quickwit/quickwit-common/src/pubsub.rs index 29fab1101cb..70683167169 100644 --- a/quickwit/quickwit-common/src/pubsub.rs +++ b/quickwit/quickwit-common/src/pubsub.rs @@ -42,7 +42,7 @@ pub trait EventSubscriber: Send + Sync + 'static { impl EventSubscriber for F where E: Event, - F: Fn(E) + Send + Sync + 'static, + F: FnMut(E) + Send + Sync + 'static, { async fn handle_event(&mut self, event: E) { (self)(event); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 388a990086b..a628ac1feff 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -94,7 +94,7 @@ impl PublishTracker { } } - fn shard_persisted(&self, shard_id: ShardId, new_position: Position) { + fn track_persisted_position(&self, shard_id: ShardId, new_position: Position) { let mut state_handle = self.state.lock().unwrap(); match state_handle.already_published.get(&shard_id) { Some(already_published_position) if new_position <= *already_published_position => { @@ -249,7 +249,7 @@ impl IngestWorkbench { if let Some(publish_tracker) = &mut self.publish_tracker { if let Some(position) = &persist_success.replication_position_inclusive { publish_tracker - .shard_persisted(persist_success.shard_id().clone(), position.clone()); + .track_persisted_position(persist_success.shard_id().clone(), position.clone()); } } self.num_successes += 1; @@ -493,9 +493,9 @@ mod tests { let shard_id_3 = ShardId::from("test-shard-3"); let shard_id_4 = ShardId::from("test-shard-3"); - tracker.shard_persisted(shard_id_1.clone(), Position::offset(42usize)); - tracker.shard_persisted(shard_id_2.clone(), Position::offset(42usize)); - tracker.shard_persisted(shard_id_3.clone(), Position::offset(42usize)); + tracker.track_persisted_position(shard_id_1.clone(), Position::offset(42usize)); + tracker.track_persisted_position(shard_id_2.clone(), Position::offset(42usize)); + tracker.track_persisted_position(shard_id_3.clone(), Position::offset(42usize)); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { @@ -524,7 +524,7 @@ mod tests { }); // persist response received after the publish event - tracker.shard_persisted(shard_id_4.clone(), Position::offset(42usize)); + tracker.track_persisted_position(shard_id_4.clone(), Position::offset(42usize)); tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) .await @@ -538,8 +538,8 @@ mod tests { let tracker = PublishTracker::new(event_broker.clone()); let shard_id_1 = ShardId::from("test-shard-1"); let position = Position::offset(42usize); - tracker.shard_persisted(shard_id_1.clone(), position.clone()); - tracker.shard_persisted(ShardId::from("test-shard-2"), position.clone()); + tracker.track_persisted_position(shard_id_1.clone(), position.clone()); + tracker.track_persisted_position(ShardId::from("test-shard-2"), position.clone()); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { From f195b910ffbdf194c37a411bee85e4281f7b0b8e Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 30 Aug 2024 11:14:48 +0200 Subject: [PATCH 16/25] Failing test showing a race condition --- .../src/ingest_v2/workbench.rs | 56 +++++++++++++------ 1 file changed, 40 insertions(+), 16 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index a628ac1feff..f77ae5e6101 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -534,26 +534,50 @@ mod tests { #[tokio::test] async fn test_publish_tracker_waits() { let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - let event_broker = EventBroker::default(); - let tracker = PublishTracker::new(event_broker.clone()); let shard_id_1 = ShardId::from("test-shard-1"); let position = Position::offset(42usize); - tracker.track_persisted_position(shard_id_1.clone(), position.clone()); - tracker.track_persisted_position(ShardId::from("test-shard-2"), position.clone()); - event_broker.publish(ShardPositionsUpdate { - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - }, - updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] - .into_iter() - .collect(), - }); + { + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + tracker.track_persisted_position(shard_id_1.clone(), position.clone()); + tracker.track_persisted_position(ShardId::from("test-shard-2"), position.clone()); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] + .into_iter() + .collect(), + }); - tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) - .await - .unwrap_err(); + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap_err(); + } + { + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + tracker.track_persisted_position(shard_id_1.clone(), position.clone()); + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] + .into_iter() + .collect(), + }); + // sleep to make sure the event is processed + tokio::time::sleep(Duration::from_millis(50)).await; + tracker.track_persisted_position(ShardId::from("test-shard-2"), position.clone()); + + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap_err(); + } } #[test] From d10d263e98050d4e57a087d9cb6d74400a6354be Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 30 Aug 2024 11:25:25 +0200 Subject: [PATCH 17/25] Fix race condition --- .../src/ingest_v2/workbench.rs | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index f77ae5e6101..70dae3488cf 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -43,16 +43,17 @@ struct PublishState { already_published: HashMap, } -/// A helper for tracking the progress of the publish events when running in -/// `wait_for` commit mode. +/// A helper for awaiting shard publish events when running in `wait_for` and +/// `force` commit mode. /// /// Registers a set of shard positions and listens to [`ShardPositionsUpdate`] /// events to assert when all the persisted events have been published. To make /// sure that no events are missed: /// - the tracker should be created before the persist requests are sent -/// - the `shard_persisted` method should for all successful persist subrequests +/// - `track_persisted_position` should be called for all successful persist subrequests struct PublishTracker { state: Arc>, + // sync::notify instead of sync::oneshot because we don't want to store the permit publish_complete: Arc, _publish_listen_handle: EventSubscriptionHandle, } @@ -73,7 +74,11 @@ impl PublishTracker { if updated_position >= shard_position { state_handle.awaiting_publish.remove(updated_shard_id); if state_handle.awaiting_publish.is_empty() { - publish_complete_notifier.notify_one(); + // The notification is only relevant once + // `self.wait_publish_complete()` is called. + // Before that, `state.awaiting_publish` might + // still be re-populated. + publish_complete_notifier.notify_waiters(); } } } else { @@ -109,13 +114,15 @@ impl PublishTracker { } async fn wait_publish_complete(self) { - // correctness: - // - `state.awaiting_publish` can only grow before this (`self` is consumed) - // - `publish_complete.notify()` is called as soon as the last shard is published + // correctness: new shards cannot be added to `state.awaiting_publish` + // at this point because `self` is consumed. By subscribing to + // `publish_complete` before checking `awaiting_publish`, we make sure we + // don't miss the moment when it becomes empty. + let notified = self.publish_complete.notified(); if self.state.lock().unwrap().awaiting_publish.is_empty() { return; } - self.publish_complete.notified().await; + notified.await; } } From 4e2ea475a597af9ebd8016158863865e398b4e10 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 30 Aug 2024 15:48:17 +0200 Subject: [PATCH 18/25] Only track shard that were requested --- .../quickwit-ingest/src/ingest_v2/router.rs | 1 + .../src/ingest_v2/workbench.rs | 220 +++++++++++++----- 2 files changed, 165 insertions(+), 56 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index b999e6e57fb..d20d5c2e74c 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -425,6 +425,7 @@ impl IngestRouter { subrequests, commit_type: commit_type as i32, }; + workbench.record_persist_request(&persist_request); let persist_future = async move { let persist_result = tokio::time::timeout( PERSIST_REQUEST_TIMEOUT, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 70dae3488cf..ef25fcc93b5 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -26,33 +26,43 @@ use quickwit_proto::control_plane::{ GetOrCreateOpenShardsFailure, GetOrCreateOpenShardsFailureReason, }; use quickwit_proto::indexing::ShardPositionsUpdate; -use quickwit_proto::ingest::ingester::{PersistFailure, PersistFailureReason, PersistSuccess}; +use quickwit_proto::ingest::ingester::{ + PersistFailure, PersistFailureReason, PersistRequest, PersistSuccess, +}; use quickwit_proto::ingest::router::{ IngestFailure, IngestFailureReason, IngestResponseV2, IngestSubrequest, IngestSuccess, }; use quickwit_proto::ingest::{IngestV2Error, RateLimitingCause}; use quickwit_proto::types::{NodeId, Position, ShardId, SubrequestId}; use tokio::sync::Notify; -use tracing::warn; +use tracing::{error, warn}; use super::router::PersistRequestSummary; +enum PublishState { + Tracked, + AwaitingPublish(Position), + Published(Position), +} + #[derive(Default)] -struct PublishState { - awaiting_publish: HashMap, - already_published: HashMap, +struct ShardPublishStates { + states: HashMap, + awaiting_count: usize, } /// A helper for awaiting shard publish events when running in `wait_for` and /// `force` commit mode. /// /// Registers a set of shard positions and listens to [`ShardPositionsUpdate`] -/// events to assert when all the persisted events have been published. To make -/// sure that no events are missed: -/// - the tracker should be created before the persist requests are sent -/// - `track_persisted_position` should be called for all successful persist subrequests +/// events to assert when all the persisted events have been published. To +/// ensure that no events are missed: +/// - create the tracker before any persist requests is sent +/// - call `register_requested_shard` before each persist request to ensure that +/// the associated publish events are recorded +/// - call `track_persisted_shard_position` after each successful persist subrequests struct PublishTracker { - state: Arc>, + state: Arc>, // sync::notify instead of sync::oneshot because we don't want to store the permit publish_complete: Arc, _publish_listen_handle: EventSubscriptionHandle, @@ -60,7 +70,7 @@ struct PublishTracker { impl PublishTracker { fn new(event_tracker: EventBroker) -> Self { - let state = Arc::new(Mutex::new(PublishState::default())); + let state = Arc::new(Mutex::new(ShardPublishStates::default())); let state_clone = state.clone(); let publish_complete = Arc::new(Notify::new()); let publish_complete_notifier = publish_complete.clone(); @@ -68,28 +78,38 @@ impl PublishTracker { event_tracker.subscribe(move |update: ShardPositionsUpdate| { let mut state_handle = state_clone.lock().unwrap(); for (updated_shard_id, updated_position) in &update.updated_shard_positions { - if let Some(shard_position) = - state_handle.awaiting_publish.get(updated_shard_id) - { - if updated_position >= shard_position { - state_handle.awaiting_publish.remove(updated_shard_id); - if state_handle.awaiting_publish.is_empty() { - // The notification is only relevant once - // `self.wait_publish_complete()` is called. - // Before that, `state.awaiting_publish` might - // still be re-populated. - publish_complete_notifier.notify_waiters(); + if let Some(publish_state) = state_handle.states.get_mut(updated_shard_id) { + match publish_state { + PublishState::AwaitingPublish(shard_position) + if updated_position >= shard_position => + { + *publish_state = PublishState::Published(updated_position.clone()); + state_handle.awaiting_count -= 1; + if state_handle.awaiting_count == 0 { + // The notification is only relevant once + // `self.wait_publish_complete()` is called. + // Before that, `state.awaiting_publish` might + // still be re-populated. + publish_complete_notifier.notify_waiters(); + } + } + PublishState::Published(current_position) + if updated_position > current_position => + { + *current_position = updated_position.clone(); + } + PublishState::Tracked => { + *publish_state = PublishState::Published(updated_position.clone()); + } + PublishState::Published(_) => { + // looks like a duplicate or out-of-order event + } + PublishState::AwaitingPublish(_) => { + // the shard made some progress but we are waiting for more } } - } else { - // Save this position update in case the publish update - // event arrived before the shard persist response. We - // might build a state that tracks irrelevant shards for - // the duration of the query but that should be fine. - state_handle - .already_published - .insert(updated_shard_id.clone(), updated_position.clone()); } + // else: this shard is not being tracked here } }); Self { @@ -99,27 +119,48 @@ impl PublishTracker { } } - fn track_persisted_position(&self, shard_id: ShardId, new_position: Position) { + fn register_requested_shard<'a>(&'a self, shard_ids: impl IntoIterator) { let mut state_handle = self.state.lock().unwrap(); - match state_handle.already_published.get(&shard_id) { - Some(already_published_position) if new_position <= *already_published_position => { - // already published, no need to track this shard's position updates - } - _ => { - state_handle - .awaiting_publish - .insert(shard_id.clone(), new_position.clone()); + for shard_id in shard_ids { + state_handle + .states + .entry(shard_id.clone()) + .or_insert(PublishState::Tracked); + } + } + + fn track_persisted_shard_position(&self, shard_id: ShardId, new_position: Position) { + let mut state_handle = self.state.lock().unwrap(); + if let Some(publish_state) = state_handle.states.get_mut(&shard_id) { + match publish_state { + PublishState::Published(current_position) if new_position <= *current_position => { + // new position already published, no need to track it + } + PublishState::AwaitingPublish(old_position) => { + error!( + %old_position, + %new_position, + %shard_id, + "shard persisted positions should not be tracked multiple times" + ); + } + PublishState::Tracked | PublishState::Published(_) => { + *publish_state = PublishState::AwaitingPublish(new_position.clone()); + state_handle.awaiting_count += 1; + } } + } else { + error!(%shard_id, "requested shards should be registered before their position is tracked") } } async fn wait_publish_complete(self) { - // correctness: new shards cannot be added to `state.awaiting_publish` - // at this point because `self` is consumed. By subscribing to - // `publish_complete` before checking `awaiting_publish`, we make sure we - // don't miss the moment when it becomes empty. + // correctness: `awaiting_count` cannot be increased after this point + // because `self` is consumed. By subscribing to `publish_complete` + // before checking `awaiting_count`, we make sure we don't miss the + // moment when it becomes 0. let notified = self.publish_complete.notified(); - if self.state.lock().unwrap().awaiting_publish.is_empty() { + if self.state.lock().unwrap().awaiting_count == 0 { return; } notified.await; @@ -222,6 +263,16 @@ impl IngestWorkbench { .all(|subworbench| !subworbench.is_pending()) } + pub fn record_persist_request(&self, persist_request: &PersistRequest) { + if let Some(publish_tracker) = &self.publish_tracker { + let shards = persist_request + .subrequests + .iter() + .map(|subrequest| subrequest.shard_id()); + publish_tracker.register_requested_shard(shards); + } + } + pub fn record_get_or_create_open_shards_failure( &mut self, open_shards_failure: GetOrCreateOpenShardsFailure, @@ -255,8 +306,10 @@ impl IngestWorkbench { }; if let Some(publish_tracker) = &mut self.publish_tracker { if let Some(position) = &persist_success.replication_position_inclusive { - publish_tracker - .track_persisted_position(persist_success.shard_id().clone(), position.clone()); + publish_tracker.track_persisted_shard_position( + persist_success.shard_id().clone(), + position.clone(), + ); } } self.num_successes += 1; @@ -485,7 +538,7 @@ impl IngestSubworkbench { mod tests { use std::time::Duration; - use quickwit_proto::ingest::ingester::PersistFailureReason; + use quickwit_proto::ingest::ingester::{PersistFailureReason, PersistSubrequest}; use quickwit_proto::types::{IndexUid, ShardId, SourceUid}; use super::*; @@ -498,11 +551,15 @@ mod tests { let shard_id_1 = ShardId::from("test-shard-1"); let shard_id_2 = ShardId::from("test-shard-2"); let shard_id_3 = ShardId::from("test-shard-3"); - let shard_id_4 = ShardId::from("test-shard-3"); + let shard_id_4 = ShardId::from("test-shard-4"); + let shard_id_5 = ShardId::from("test-shard-5"); // not tracked - tracker.track_persisted_position(shard_id_1.clone(), Position::offset(42usize)); - tracker.track_persisted_position(shard_id_2.clone(), Position::offset(42usize)); - tracker.track_persisted_position(shard_id_3.clone(), Position::offset(42usize)); + tracker + .register_requested_shard([&shard_id_1, &shard_id_2, &shard_id_3, &shard_id_4].clone()); + + tracker.track_persisted_shard_position(shard_id_1.clone(), Position::offset(42usize)); + tracker.track_persisted_shard_position(shard_id_2.clone(), Position::offset(42usize)); + tracker.track_persisted_shard_position(shard_id_3.clone(), Position::offset(42usize)); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { @@ -512,6 +569,7 @@ mod tests { updated_shard_positions: vec![ (shard_id_1.clone(), Position::offset(42usize)), (shard_id_2.clone(), Position::offset(666usize)), + (shard_id_5.clone(), Position::offset(888usize)), ] .into_iter() .collect(), @@ -531,7 +589,7 @@ mod tests { }); // persist response received after the publish event - tracker.track_persisted_position(shard_id_4.clone(), Position::offset(42usize)); + tracker.track_persisted_shard_position(shard_id_4.clone(), Position::offset(42usize)); tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) .await @@ -542,13 +600,15 @@ mod tests { async fn test_publish_tracker_waits() { let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); let position = Position::offset(42usize); { let event_broker = EventBroker::default(); let tracker = PublishTracker::new(event_broker.clone()); - tracker.track_persisted_position(shard_id_1.clone(), position.clone()); - tracker.track_persisted_position(ShardId::from("test-shard-2"), position.clone()); + tracker.register_requested_shard([&shard_id_1, &shard_id_2].clone()); + tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); + tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { @@ -567,7 +627,8 @@ mod tests { { let event_broker = EventBroker::default(); let tracker = PublishTracker::new(event_broker.clone()); - tracker.track_persisted_position(shard_id_1.clone(), position.clone()); + tracker.register_requested_shard([&shard_id_1, &shard_id_2].clone()); + tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { index_uid: index_uid.clone(), @@ -579,7 +640,7 @@ mod tests { }); // sleep to make sure the event is processed tokio::time::sleep(Duration::from_millis(50)).await; - tracker.track_persisted_position(ShardId::from("test-shard-2"), position.clone()); + tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) .await @@ -740,6 +801,23 @@ mod tests { assert_eq!(pending_subrequests(&workbench.subworkbenches).count(), 2); assert!(!workbench.is_complete()); + let persist_request = PersistRequest { + subrequests: vec![ + PersistSubrequest { + subrequest_id: 0, + shard_id: Some(shard_id_1.clone()), + ..Default::default() + }, + PersistSubrequest { + subrequest_id: 1, + shard_id: Some(shard_id_2.clone()), + ..Default::default() + }, + ], + ..Default::default() + }; + workbench.record_persist_request(&persist_request); + let persist_success = PersistSuccess { subrequest_id: 0, shard_id: Some(shard_id_1.clone()), @@ -761,6 +839,19 @@ mod tests { replication_position_inclusive: Some(Position::offset(66usize)), ..Default::default() }; + + // retry to persist shard 2 + + let persist_request = PersistRequest { + subrequests: vec![PersistSubrequest { + subrequest_id: 1, + shard_id: Some(shard_id_2.clone()), + ..Default::default() + }], + ..Default::default() + }; + workbench.record_persist_request(&persist_request); + workbench.record_persist_success(persist_success); assert!(workbench.is_complete()); @@ -803,6 +894,23 @@ mod tests { let mut workbench = IngestWorkbench::new_with_publish_tracking(ingest_subrequests, 1, event_broker.clone()); + let persist_request = PersistRequest { + subrequests: vec![ + PersistSubrequest { + subrequest_id: 0, + shard_id: Some(shard_id_1.clone()), + ..Default::default() + }, + PersistSubrequest { + subrequest_id: 1, + shard_id: Some(shard_id_2.clone()), + ..Default::default() + }, + ], + ..Default::default() + }; + workbench.record_persist_request(&persist_request); + let persist_success = PersistSuccess { subrequest_id: 0, shard_id: Some(shard_id_1.clone()), From 95686f429224080ce5d040fa19aede954cfe6b59 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 30 Aug 2024 15:55:20 +0200 Subject: [PATCH 19/25] Fix clippy lints and typo --- quickwit/quickwit-ingest/src/ingest_v2/workbench.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index ef25fcc93b5..23c4790f7e6 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -58,7 +58,7 @@ struct ShardPublishStates { /// events to assert when all the persisted events have been published. To /// ensure that no events are missed: /// - create the tracker before any persist requests is sent -/// - call `register_requested_shard` before each persist request to ensure that +/// - call `register_requested_shards` before each persist request to ensure that /// the associated publish events are recorded /// - call `track_persisted_shard_position` after each successful persist subrequests struct PublishTracker { @@ -119,7 +119,7 @@ impl PublishTracker { } } - fn register_requested_shard<'a>(&'a self, shard_ids: impl IntoIterator) { + fn register_requested_shards<'a>(&'a self, shard_ids: impl IntoIterator) { let mut state_handle = self.state.lock().unwrap(); for shard_id in shard_ids { state_handle @@ -269,7 +269,7 @@ impl IngestWorkbench { .subrequests .iter() .map(|subrequest| subrequest.shard_id()); - publish_tracker.register_requested_shard(shards); + publish_tracker.register_requested_shards(shards); } } @@ -554,8 +554,7 @@ mod tests { let shard_id_4 = ShardId::from("test-shard-4"); let shard_id_5 = ShardId::from("test-shard-5"); // not tracked - tracker - .register_requested_shard([&shard_id_1, &shard_id_2, &shard_id_3, &shard_id_4].clone()); + tracker.register_requested_shards([&shard_id_1, &shard_id_2, &shard_id_3, &shard_id_4]); tracker.track_persisted_shard_position(shard_id_1.clone(), Position::offset(42usize)); tracker.track_persisted_shard_position(shard_id_2.clone(), Position::offset(42usize)); @@ -606,7 +605,7 @@ mod tests { { let event_broker = EventBroker::default(); let tracker = PublishTracker::new(event_broker.clone()); - tracker.register_requested_shard([&shard_id_1, &shard_id_2].clone()); + tracker.register_requested_shards([&shard_id_1, &shard_id_2]); tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); @@ -627,7 +626,7 @@ mod tests { { let event_broker = EventBroker::default(); let tracker = PublishTracker::new(event_broker.clone()); - tracker.register_requested_shard([&shard_id_1, &shard_id_2].clone()); + tracker.register_requested_shards([&shard_id_1, &shard_id_2]); tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); event_broker.publish(ShardPositionsUpdate { source_uid: SourceUid { From dd0fa117c86265806bc1a8ade09ca45c864c0c92 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 30 Aug 2024 16:35:49 +0200 Subject: [PATCH 20/25] Describe publish states --- quickwit/quickwit-ingest/src/ingest_v2/workbench.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 23c4790f7e6..b181cb6a47a 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -40,8 +40,13 @@ use tracing::{error, warn}; use super::router::PersistRequestSummary; enum PublishState { + /// The persist request for this shard has been sent Tracked, + /// The persist request for this shard success response has been received + /// but the position has not yet been published AwaitingPublish(Position), + /// The shard has been published up to this position (might happen before + /// the persist success is received) Published(Position), } From e812be804293563e0f32984dff14e3a1e7569c87 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 2 Sep 2024 14:20:10 +0200 Subject: [PATCH 21/25] Refacto to separate module --- quickwit/quickwit-ingest/src/ingest_v2/mod.rs | 1 + .../src/ingest_v2/publish_tracker.rs | 292 ++++++++++++++++++ .../src/ingest_v2/workbench.rs | 252 +-------------- 3 files changed, 300 insertions(+), 245 deletions(-) create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index 9ff314919e8..381be88535e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -27,6 +27,7 @@ mod metrics; mod models; mod mrecord; mod mrecordlog_utils; +mod publish_tracker; mod rate_meter; mod replication; mod router; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs b/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs new file mode 100644 index 00000000000..bf826c4e107 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs @@ -0,0 +1,292 @@ +// 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::HashMap; +use std::sync::{Arc, Mutex}; + +use quickwit_common::pubsub::{EventBroker, EventSubscriptionHandle}; +use quickwit_proto::indexing::ShardPositionsUpdate; +use quickwit_proto::types::{Position, ShardId}; +use tokio::sync::Notify; +use tracing::error; + +/// A helper for awaiting shard publish events when running in `wait_for` and +/// `force` commit mode. +/// +/// Registers a set of shard positions and listens to [`ShardPositionsUpdate`] +/// events to assert when all the persisted events have been published. To +/// ensure that no events are missed: +/// - create the tracker before any persist requests is sent +/// - call `register_requested_shards` before each persist request to ensure that +/// the associated publish events are recorded +/// - call `track_persisted_shard_position` after each successful persist subrequests +pub struct PublishTracker { + state: Arc>, + // sync::notify instead of sync::oneshot because we don't want to store the permit + publish_complete: Arc, + _publish_listen_handle: EventSubscriptionHandle, +} + +impl PublishTracker { + pub fn new(event_tracker: EventBroker) -> Self { + let state = Arc::new(Mutex::new(ShardPublishStates::default())); + let state_clone = state.clone(); + let publish_complete = Arc::new(Notify::new()); + let publish_complete_notifier = publish_complete.clone(); + let _publish_listen_handle = + event_tracker.subscribe(move |update: ShardPositionsUpdate| { + let mut publish_states = state_clone.lock().unwrap(); + for (updated_shard_id, updated_position) in &update.updated_shard_positions { + publish_states.position_published( + updated_shard_id, + updated_position, + &publish_complete_notifier, + ); + } + }); + Self { + state, + _publish_listen_handle, + publish_complete, + } + } + + pub fn register_requested_shards<'a>( + &'a self, + shard_ids: impl IntoIterator, + ) { + let mut publish_states = self.state.lock().unwrap(); + for shard_id in shard_ids { + publish_states.shard_tracked(shard_id.clone()); + } + } + + pub fn track_persisted_shard_position(&self, shard_id: ShardId, new_position: Position) { + let mut publish_states = self.state.lock().unwrap(); + publish_states.position_persisted(&shard_id, &new_position) + } + + pub async fn wait_publish_complete(self) { + // correctness: `awaiting_count` cannot be increased after this point + // because `self` is consumed. By subscribing to `publish_complete` + // before checking `awaiting_count`, we make sure we don't miss the + // moment when it becomes 0. + let notified = self.publish_complete.notified(); + if self.state.lock().unwrap().awaiting_count == 0 { + return; + } + notified.await; + } +} + +enum PublishState { + /// The persist request for this shard has been sent + Tracked, + /// The persist request for this shard success response has been received + /// but the position has not yet been published + AwaitingPublish(Position), + /// The shard has been published up to this position (might happen before + /// the persist success is received) + Published(Position), +} + +#[derive(Default)] +struct ShardPublishStates { + states: HashMap, + awaiting_count: usize, +} + +impl ShardPublishStates { + fn shard_tracked(&mut self, shard_id: ShardId) { + self.states.entry(shard_id).or_insert(PublishState::Tracked); + } + + fn position_published( + &mut self, + shard_id: &ShardId, + new_position: &Position, + publish_complete_notifier: &Notify, + ) { + if let Some(publish_state) = self.states.get_mut(shard_id) { + match publish_state { + PublishState::AwaitingPublish(shard_position) if new_position >= shard_position => { + *publish_state = PublishState::Published(new_position.clone()); + self.awaiting_count -= 1; + if self.awaiting_count == 0 { + // The notification is only relevant once + // `self.wait_publish_complete()` is called. + // Before that, `state.awaiting_publish` might + // still be re-populated. + publish_complete_notifier.notify_waiters(); + } + } + PublishState::Published(current_position) if new_position > current_position => { + *current_position = new_position.clone(); + } + PublishState::Tracked => { + *publish_state = PublishState::Published(new_position.clone()); + } + PublishState::Published(_) => { + // looks like a duplicate or out-of-order event + } + PublishState::AwaitingPublish(_) => { + // the shard made some progress but we are waiting for more + } + } + } + // else: this shard is not being tracked here + } + + fn position_persisted(&mut self, shard_id: &ShardId, new_position: &Position) { + if let Some(publish_state) = self.states.get_mut(shard_id) { + match publish_state { + PublishState::Published(current_position) if new_position <= current_position => { + // new position already published, no need to track it + } + PublishState::AwaitingPublish(old_position) => { + error!( + %old_position, + %new_position, + %shard_id, + "shard persisted positions should not be tracked multiple times" + ); + } + PublishState::Tracked | PublishState::Published(_) => { + *publish_state = PublishState::AwaitingPublish(new_position.clone()); + self.awaiting_count += 1; + } + } + } else { + error!(%shard_id, "requested shards should be registered before their position is tracked") + } + } +} + +#[cfg(test)] +mod tests { + use std::time::Duration; + + use quickwit_proto::types::{IndexUid, ShardId, SourceUid}; + + use super::*; + + #[tokio::test] + async fn test_publish_tracker() { + let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); + let shard_id_3 = ShardId::from("test-shard-3"); + let shard_id_4 = ShardId::from("test-shard-4"); + let shard_id_5 = ShardId::from("test-shard-5"); // not tracked + + tracker.register_requested_shards([&shard_id_1, &shard_id_2, &shard_id_3, &shard_id_4]); + + tracker.track_persisted_shard_position(shard_id_1.clone(), Position::offset(42usize)); + tracker.track_persisted_shard_position(shard_id_2.clone(), Position::offset(42usize)); + tracker.track_persisted_shard_position(shard_id_3.clone(), Position::offset(42usize)); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![ + (shard_id_1.clone(), Position::offset(42usize)), + (shard_id_2.clone(), Position::offset(666usize)), + (shard_id_5.clone(), Position::offset(888usize)), + ] + .into_iter() + .collect(), + }); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![ + (shard_id_3.clone(), Position::eof(42usize)), + (shard_id_4.clone(), Position::offset(42usize)), + ] + .into_iter() + .collect(), + }); + + // persist response received after the publish event + tracker.track_persisted_shard_position(shard_id_4.clone(), Position::offset(42usize)); + + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap(); + } + + #[tokio::test] + async fn test_publish_tracker_waits() { + let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); + let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); + let position = Position::offset(42usize); + + { + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + tracker.register_requested_shards([&shard_id_1, &shard_id_2]); + tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); + tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); + + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] + .into_iter() + .collect(), + }); + + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap_err(); + } + { + let event_broker = EventBroker::default(); + let tracker = PublishTracker::new(event_broker.clone()); + tracker.register_requested_shards([&shard_id_1, &shard_id_2]); + tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); + event_broker.publish(ShardPositionsUpdate { + source_uid: SourceUid { + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + }, + updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] + .into_iter() + .collect(), + }); + // sleep to make sure the event is processed + tokio::time::sleep(Duration::from_millis(50)).await; + tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); + + tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) + .await + .unwrap_err(); + } + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index b181cb6a47a..7dab68c5485 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -17,15 +17,13 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::{BTreeMap, HashMap, HashSet}; -use std::sync::{Arc, Mutex}; +use std::collections::{BTreeMap, HashSet}; -use quickwit_common::pubsub::{EventBroker, EventSubscriptionHandle}; +use quickwit_common::pubsub::EventBroker; use quickwit_common::rate_limited_error; use quickwit_proto::control_plane::{ GetOrCreateOpenShardsFailure, GetOrCreateOpenShardsFailureReason, }; -use quickwit_proto::indexing::ShardPositionsUpdate; use quickwit_proto::ingest::ingester::{ PersistFailure, PersistFailureReason, PersistRequest, PersistSuccess, }; @@ -33,145 +31,12 @@ use quickwit_proto::ingest::router::{ IngestFailure, IngestFailureReason, IngestResponseV2, IngestSubrequest, IngestSuccess, }; use quickwit_proto::ingest::{IngestV2Error, RateLimitingCause}; -use quickwit_proto::types::{NodeId, Position, ShardId, SubrequestId}; -use tokio::sync::Notify; -use tracing::{error, warn}; +use quickwit_proto::types::{NodeId, ShardId, SubrequestId}; +use tracing::warn; +use super::publish_tracker::PublishTracker; use super::router::PersistRequestSummary; -enum PublishState { - /// The persist request for this shard has been sent - Tracked, - /// The persist request for this shard success response has been received - /// but the position has not yet been published - AwaitingPublish(Position), - /// The shard has been published up to this position (might happen before - /// the persist success is received) - Published(Position), -} - -#[derive(Default)] -struct ShardPublishStates { - states: HashMap, - awaiting_count: usize, -} - -/// A helper for awaiting shard publish events when running in `wait_for` and -/// `force` commit mode. -/// -/// Registers a set of shard positions and listens to [`ShardPositionsUpdate`] -/// events to assert when all the persisted events have been published. To -/// ensure that no events are missed: -/// - create the tracker before any persist requests is sent -/// - call `register_requested_shards` before each persist request to ensure that -/// the associated publish events are recorded -/// - call `track_persisted_shard_position` after each successful persist subrequests -struct PublishTracker { - state: Arc>, - // sync::notify instead of sync::oneshot because we don't want to store the permit - publish_complete: Arc, - _publish_listen_handle: EventSubscriptionHandle, -} - -impl PublishTracker { - fn new(event_tracker: EventBroker) -> Self { - let state = Arc::new(Mutex::new(ShardPublishStates::default())); - let state_clone = state.clone(); - let publish_complete = Arc::new(Notify::new()); - let publish_complete_notifier = publish_complete.clone(); - let _publish_listen_handle = - event_tracker.subscribe(move |update: ShardPositionsUpdate| { - let mut state_handle = state_clone.lock().unwrap(); - for (updated_shard_id, updated_position) in &update.updated_shard_positions { - if let Some(publish_state) = state_handle.states.get_mut(updated_shard_id) { - match publish_state { - PublishState::AwaitingPublish(shard_position) - if updated_position >= shard_position => - { - *publish_state = PublishState::Published(updated_position.clone()); - state_handle.awaiting_count -= 1; - if state_handle.awaiting_count == 0 { - // The notification is only relevant once - // `self.wait_publish_complete()` is called. - // Before that, `state.awaiting_publish` might - // still be re-populated. - publish_complete_notifier.notify_waiters(); - } - } - PublishState::Published(current_position) - if updated_position > current_position => - { - *current_position = updated_position.clone(); - } - PublishState::Tracked => { - *publish_state = PublishState::Published(updated_position.clone()); - } - PublishState::Published(_) => { - // looks like a duplicate or out-of-order event - } - PublishState::AwaitingPublish(_) => { - // the shard made some progress but we are waiting for more - } - } - } - // else: this shard is not being tracked here - } - }); - Self { - state, - _publish_listen_handle, - publish_complete, - } - } - - fn register_requested_shards<'a>(&'a self, shard_ids: impl IntoIterator) { - let mut state_handle = self.state.lock().unwrap(); - for shard_id in shard_ids { - state_handle - .states - .entry(shard_id.clone()) - .or_insert(PublishState::Tracked); - } - } - - fn track_persisted_shard_position(&self, shard_id: ShardId, new_position: Position) { - let mut state_handle = self.state.lock().unwrap(); - if let Some(publish_state) = state_handle.states.get_mut(&shard_id) { - match publish_state { - PublishState::Published(current_position) if new_position <= *current_position => { - // new position already published, no need to track it - } - PublishState::AwaitingPublish(old_position) => { - error!( - %old_position, - %new_position, - %shard_id, - "shard persisted positions should not be tracked multiple times" - ); - } - PublishState::Tracked | PublishState::Published(_) => { - *publish_state = PublishState::AwaitingPublish(new_position.clone()); - state_handle.awaiting_count += 1; - } - } - } else { - error!(%shard_id, "requested shards should be registered before their position is tracked") - } - } - - async fn wait_publish_complete(self) { - // correctness: `awaiting_count` cannot be increased after this point - // because `self` is consumed. By subscribing to `publish_complete` - // before checking `awaiting_count`, we make sure we don't miss the - // moment when it becomes 0. - let notified = self.publish_complete.notified(); - if self.state.lock().unwrap().awaiting_count == 0 { - return; - } - notified.await; - } -} - /// A helper struct for managing the state of the subrequests of an ingest request during multiple /// persist attempts. #[derive(Default)] @@ -543,115 +408,12 @@ impl IngestSubworkbench { mod tests { use std::time::Duration; + use quickwit_proto::indexing::ShardPositionsUpdate; use quickwit_proto::ingest::ingester::{PersistFailureReason, PersistSubrequest}; - use quickwit_proto::types::{IndexUid, ShardId, SourceUid}; + use quickwit_proto::types::{IndexUid, Position, ShardId, SourceUid}; use super::*; - #[tokio::test] - async fn test_publish_tracker() { - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - let event_broker = EventBroker::default(); - let tracker = PublishTracker::new(event_broker.clone()); - let shard_id_1 = ShardId::from("test-shard-1"); - let shard_id_2 = ShardId::from("test-shard-2"); - let shard_id_3 = ShardId::from("test-shard-3"); - let shard_id_4 = ShardId::from("test-shard-4"); - let shard_id_5 = ShardId::from("test-shard-5"); // not tracked - - tracker.register_requested_shards([&shard_id_1, &shard_id_2, &shard_id_3, &shard_id_4]); - - tracker.track_persisted_shard_position(shard_id_1.clone(), Position::offset(42usize)); - tracker.track_persisted_shard_position(shard_id_2.clone(), Position::offset(42usize)); - tracker.track_persisted_shard_position(shard_id_3.clone(), Position::offset(42usize)); - - event_broker.publish(ShardPositionsUpdate { - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - }, - updated_shard_positions: vec![ - (shard_id_1.clone(), Position::offset(42usize)), - (shard_id_2.clone(), Position::offset(666usize)), - (shard_id_5.clone(), Position::offset(888usize)), - ] - .into_iter() - .collect(), - }); - - event_broker.publish(ShardPositionsUpdate { - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - }, - updated_shard_positions: vec![ - (shard_id_3.clone(), Position::eof(42usize)), - (shard_id_4.clone(), Position::offset(42usize)), - ] - .into_iter() - .collect(), - }); - - // persist response received after the publish event - tracker.track_persisted_shard_position(shard_id_4.clone(), Position::offset(42usize)); - - tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) - .await - .unwrap(); - } - - #[tokio::test] - async fn test_publish_tracker_waits() { - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - let shard_id_1 = ShardId::from("test-shard-1"); - let shard_id_2 = ShardId::from("test-shard-2"); - let position = Position::offset(42usize); - - { - let event_broker = EventBroker::default(); - let tracker = PublishTracker::new(event_broker.clone()); - tracker.register_requested_shards([&shard_id_1, &shard_id_2]); - tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); - tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); - - event_broker.publish(ShardPositionsUpdate { - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - }, - updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] - .into_iter() - .collect(), - }); - - tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) - .await - .unwrap_err(); - } - { - let event_broker = EventBroker::default(); - let tracker = PublishTracker::new(event_broker.clone()); - tracker.register_requested_shards([&shard_id_1, &shard_id_2]); - tracker.track_persisted_shard_position(shard_id_1.clone(), position.clone()); - event_broker.publish(ShardPositionsUpdate { - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - }, - updated_shard_positions: vec![(shard_id_1.clone(), position.clone())] - .into_iter() - .collect(), - }); - // sleep to make sure the event is processed - tokio::time::sleep(Duration::from_millis(50)).await; - tracker.track_persisted_shard_position(shard_id_2.clone(), position.clone()); - - tokio::time::timeout(Duration::from_millis(200), tracker.wait_publish_complete()) - .await - .unwrap_err(); - } - } - #[test] fn test_ingest_subworkbench() { let subrequest = IngestSubrequest { From 7cfc7337994416dda58c7e2a83fd1f43754a6aa5 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 2 Sep 2024 15:05:48 +0200 Subject: [PATCH 22/25] Add independent tests on internal state --- .../src/ingest_v2/publish_tracker.rs | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs b/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs index bf826c4e107..64f9c907722 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/publish_tracker.rs @@ -186,6 +186,55 @@ mod tests { use super::*; + #[tokio::test] + async fn test_shard_publish_states() { + let mut shard_publish_states = ShardPublishStates::default(); + let notifier = Arc::new(Notify::new()); + + let shard_id_1 = ShardId::from("test-shard-1"); + let shard_id_2 = ShardId::from("test-shard-2"); + let shard_id_3 = ShardId::from("test-shard-3"); + let shard_id_4 = ShardId::from("test-shard-4"); // not tracked + + shard_publish_states.shard_tracked(shard_id_1.clone()); + shard_publish_states.shard_tracked(shard_id_2.clone()); + shard_publish_states.shard_tracked(shard_id_3.clone()); + + let notifier_receiver = notifier.clone(); + let notified_subscription = notifier_receiver.notified(); + + shard_publish_states.position_persisted(&shard_id_1, &Position::offset(10usize)); + assert_eq!(shard_publish_states.awaiting_count, 1); + shard_publish_states.position_persisted(&shard_id_2, &Position::offset(20usize)); + assert_eq!(shard_publish_states.awaiting_count, 2); + shard_publish_states.position_published(&shard_id_1, &Position::offset(15usize), ¬ifier); + assert_eq!(shard_publish_states.awaiting_count, 1); + shard_publish_states.position_published(&shard_id_2, &Position::offset(20usize), ¬ifier); + assert_eq!(shard_publish_states.awaiting_count, 0); + + // check that only the notification that was subscribed before holds a permit + tokio::time::timeout(Duration::from_millis(100), notifier.notified()) + .await + .unwrap_err(); + tokio::time::timeout(Duration::from_millis(100), notified_subscription) + .await + .unwrap(); + + let notified_subscription = notifier_receiver.notified(); + shard_publish_states.position_published(&shard_id_3, &Position::offset(10usize), ¬ifier); + assert_eq!(shard_publish_states.awaiting_count, 0); + shard_publish_states.position_persisted(&shard_id_3, &Position::offset(10usize)); + assert_eq!(shard_publish_states.awaiting_count, 0); + // no notification expected here as the shard never becomes AwaitingPublish + tokio::time::timeout(Duration::from_millis(100), notified_subscription) + .await + .unwrap_err(); + // shard 4 is not tracked + shard_publish_states.position_published(&shard_id_4, &Position::offset(10usize), ¬ifier); + assert_eq!(shard_publish_states.awaiting_count, 0); + assert!(!shard_publish_states.states.contains_key(&shard_id_4)); + } + #[tokio::test] async fn test_publish_tracker() { let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); From 779f0204a30b172431a5715d59910c10c5a7dc93 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 2 Sep 2024 16:13:18 +0200 Subject: [PATCH 23/25] Add shard pruning endpoint (#5335) --- .../src/metastore/control_plane_metastore.rs | 13 +- .../file_backed/file_backed_index/mod.rs | 11 +- .../file_backed/file_backed_index/shards.rs | 122 ++++++++- .../src/metastore/file_backed/mod.rs | 17 +- .../src/metastore/postgres/metastore.rs | 41 ++- .../postgres/queries/shards/prune_age.sql | 4 + .../postgres/queries/shards/prune_count.sql | 12 + quickwit/quickwit-metastore/src/tests/mod.rs | 6 + .../quickwit-metastore/src/tests/shard.rs | 128 ++++++++- .../protos/quickwit/metastore.proto | 17 ++ .../codegen/quickwit/quickwit.metastore.rs | 258 ++++++++++++++++++ quickwit/quickwit-proto/src/getters.rs | 2 + 12 files changed, 620 insertions(+), 11 deletions(-) create mode 100644 quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_age.sql create mode 100644 quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_count.sql diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index c7183acfbd0..80a576219e1 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -35,8 +35,9 @@ use quickwit_proto::metastore::{ ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, OpenShardsRequest, OpenShardsResponse, - PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, - UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + PruneShardsRequest, PruneShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, + StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, }; /// A [`MetastoreService`] implementation that proxies some requests to the control plane so it can @@ -236,6 +237,14 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.delete_shards(request).await } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> MetastoreResult { + // TODO this call should go through the control plane which should apply debounce + self.metastore.prune_shards(request).await + } + // Index Template API async fn create_index_template( diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs index d34a6ea781b..b1bc18d3f71 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs @@ -36,7 +36,8 @@ use quickwit_config::{ use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, EntityKind, ListShardsSubrequest, ListShardsSubresponse, - MetastoreError, MetastoreResult, OpenShardSubrequest, OpenShardSubresponse, + MetastoreError, MetastoreResult, OpenShardSubrequest, OpenShardSubresponse, PruneShardsRequest, + PruneShardsResponse, }; use quickwit_proto::types::{IndexUid, PublishToken, SourceId, SplitId}; use serde::{Deserialize, Serialize}; @@ -653,6 +654,14 @@ impl FileBackedIndex { .delete_shards(request) } + pub(crate) fn prune_shards( + &mut self, + request: PruneShardsRequest, + ) -> MetastoreResult> { + self.get_shards_for_source_mut(&request.source_id)? + .prune_shards(request) + } + pub(crate) fn list_shards( &self, subrequest: ListShardsSubrequest, diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/shards.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/shards.rs index eb7bb3d013b..ac93f9efc9d 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/shards.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/shards.rs @@ -26,7 +26,7 @@ use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, DeleteShardsRequest, DeleteShardsResponse, EntityKind, ListShardsSubrequest, ListShardsSubresponse, MetastoreError, MetastoreResult, - OpenShardSubrequest, OpenShardSubresponse, + OpenShardSubrequest, OpenShardSubresponse, PruneShardsRequest, PruneShardsResponse, }; use quickwit_proto::types::{queue_id, IndexUid, Position, PublishToken, ShardId, SourceId}; use time::OffsetDateTime; @@ -239,6 +239,45 @@ impl Shards { } } + pub(super) fn prune_shards( + &mut self, + request: PruneShardsRequest, + ) -> MetastoreResult> { + let initial_shard_count = self.shards.len(); + + if let Some(max_age) = request.max_age { + self.shards.retain(|_, shard| { + let limit_timestamp = OffsetDateTime::now_utc().unix_timestamp() - max_age as i64; + shard.update_timestamp >= limit_timestamp + }); + }; + if let Some(max_count) = request.max_count { + let max_count = max_count as usize; + if max_count < self.shards.len() { + let num_to_remove = self.shards.len() - max_count; + let shard_ids_to_delete = self + .shards + .values() + .sorted_by_key(|shard| shard.update_timestamp) + .take(num_to_remove) + .map(|shard| shard.shard_id().clone()) + .collect_vec(); + for shard_id in shard_ids_to_delete { + self.shards.remove(&shard_id); + } + } + } + let response = PruneShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + }; + if initial_shard_count > self.shards.len() { + Ok(MutationOccurred::Yes(response)) + } else { + Ok(MutationOccurred::No(response)) + } + } + pub(super) fn list_shards( &self, subrequest: ListShardsSubrequest, @@ -594,4 +633,85 @@ mod tests { assert!(shards.shards.is_empty()); } + + #[test] + fn test_prune_shards() { + let index_uid = IndexUid::for_test("test-index", 0); + let source_id = "test-source".to_string(); + let mut shards = Shards::empty(index_uid.clone(), source_id.clone()); + + let request = PruneShardsRequest { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + max_age: None, + max_count: None, + }; + let MutationOccurred::No(response) = shards.prune_shards(request).unwrap() else { + panic!("expected `MutationOccurred::No`"); + }; + assert_eq!(response.index_uid(), &index_uid); + assert_eq!(response.source_id, source_id); + + let request = PruneShardsRequest { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + max_age: Some(50), + max_count: None, + }; + let MutationOccurred::No(response) = shards.prune_shards(request).unwrap() else { + panic!("expected `MutationOccurred::No`"); + }; + assert_eq!(response.index_uid(), &index_uid); + assert_eq!(response.source_id, source_id); + + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + shards.shards.insert( + ShardId::from(0), + Shard { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + shard_id: Some(ShardId::from(0)), + shard_state: ShardState::Open as i32, + publish_position_inclusive: Some(Position::eof(0u64)), + update_timestamp: current_timestamp - 200, + ..Default::default() + }, + ); + shards.shards.insert( + ShardId::from(1), + Shard { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + shard_id: Some(ShardId::from(1)), + shard_state: ShardState::Open as i32, + publish_position_inclusive: Some(Position::offset(0u64)), + update_timestamp: current_timestamp - 100, + ..Default::default() + }, + ); + + let request = PruneShardsRequest { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + max_age: Some(150), + max_count: None, + }; + let MutationOccurred::Yes(response) = shards.prune_shards(request).unwrap() else { + panic!("expected `MutationOccurred::Yes`"); + }; + assert_eq!(response.index_uid(), &index_uid); + assert_eq!(response.source_id, source_id); + + let request = PruneShardsRequest { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + max_age: Some(150), + max_count: None, + }; + let MutationOccurred::No(response) = shards.prune_shards(request).unwrap() else { + panic!("expected `MutationOccurred::No`"); + }; + assert_eq!(response.index_uid(), &index_uid); + assert_eq!(response.source_id, source_id); + } } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs index 9780cbc386e..673c3a6be37 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs @@ -58,9 +58,9 @@ use quickwit_proto::metastore::{ ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardsRequest, - OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, - ToggleSourceRequest, UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, - UpdateSplitsDeleteOpstampResponse, + OpenShardsResponse, PruneShardsRequest, PruneShardsResponse, PublishSplitsRequest, + ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, + UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_storage::Storage; @@ -892,6 +892,17 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> MetastoreResult { + let index_uid = request.index_uid().clone(); + let response = self + .mutate(&index_uid, |index| index.prune_shards(request)) + .await?; + Ok(response) + } + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index 4ed4536f8c9..20064cae706 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use std::fmt::{self, Write}; +use std::time::Duration; use async_trait::async_trait; use futures::StreamExt; @@ -44,9 +45,10 @@ use quickwit_proto::metastore::{ ListShardsRequest, ListShardsResponse, ListShardsSubresponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, - OpenShardSubresponse, OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, - ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, - UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + OpenShardSubresponse, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, + PruneShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, + ToggleSourceRequest, UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::{IndexId, IndexUid, Position, PublishToken, ShardId, SourceId}; use sea_query::{Alias, Asterisk, Expr, Func, PostgresQueryBuilder, Query, UnionType}; @@ -1486,6 +1488,39 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> MetastoreResult { + const PRUNE_AGE_SHARDS_QUERY: &str = include_str!("queries/shards/prune_age.sql"); + const PRUNE_COUNT_SHARDS_QUERY: &str = include_str!("queries/shards/prune_count.sql"); + + if let Some(max_age) = request.max_age { + let limit_datetime = OffsetDateTime::now_utc() - Duration::from_secs(max_age as u64); + sqlx::query(PRUNE_AGE_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(limit_datetime) + .execute(&self.connection_pool) + .await?; + } + + if let Some(max_count) = request.max_count { + sqlx::query(PRUNE_COUNT_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(max_count as i64) + .execute(&self.connection_pool) + .await?; + } + + let response = PruneShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + }; + Ok(response) + } + // Index Template API async fn create_index_template( diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_age.sql b/quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_age.sql new file mode 100644 index 00000000000..65f2b7bedaf --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_age.sql @@ -0,0 +1,4 @@ +DELETE FROM shards +WHERE index_uid = $1 + AND source_id = $2 + AND update_timestamp < $3 diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_count.sql b/quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_count.sql new file mode 100644 index 00000000000..93be36bfe38 --- /dev/null +++ b/quickwit/quickwit-metastore/src/metastore/postgres/queries/shards/prune_count.sql @@ -0,0 +1,12 @@ +WITH recent_shards AS ( + SELECT shard_id + FROM shards + WHERE index_uid = $1 + AND source_id = $2 + ORDER BY update_timestamp DESC + LIMIT $3 +) +DELETE FROM shards +WHERE index_uid = $1 + AND source_id = $2 + AND shard_id NOT IN (SELECT shard_id FROM recent_shards) diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index 7c4660c9fe5..3e0add028df 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -474,6 +474,12 @@ macro_rules! metastore_test_suite { #[tokio::test] #[serial_test::file_serial] + async fn test_metastore_prune_shards() { + $crate::tests::shard::test_metastore_prune_shards::<$metastore_type>().await; + } + + #[tokio::test] + #[serial_test::serial] async fn test_metastore_apply_checkpoint_delta_v2_single_shard() { $crate::tests::shard::test_metastore_apply_checkpoint_delta_v2_single_shard::<$metastore_type>().await; } diff --git a/quickwit/quickwit-metastore/src/tests/shard.rs b/quickwit/quickwit-metastore/src/tests/shard.rs index 8b53d3901dd..80549742118 100644 --- a/quickwit/quickwit-metastore/src/tests/shard.rs +++ b/quickwit/quickwit-metastore/src/tests/shard.rs @@ -18,6 +18,7 @@ // along with this program. If not, see . use async_trait::async_trait; +use itertools::Itertools; use quickwit_common::rand::append_random_suffix; use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_proto::compatibility_shard_update_timestamp; @@ -25,9 +26,10 @@ use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ AcquireShardsRequest, AddSourceRequest, CreateIndexRequest, DeleteShardsRequest, EntityKind, ListShardsRequest, ListShardsSubrequest, MetastoreError, MetastoreService, OpenShardSubrequest, - OpenShardsRequest, PublishSplitsRequest, + OpenShardsRequest, PruneShardsRequest, PublishSplitsRequest, }; use quickwit_proto::types::{DocMappingUid, IndexUid, Position, ShardId, SourceId}; +use time::OffsetDateTime; use super::DefaultForTest; use crate::checkpoint::{IndexCheckpointDelta, PartitionId, SourceCheckpointDelta}; @@ -611,6 +613,130 @@ pub async fn test_metastore_delete_shards< cleanup_index(&mut metastore, test_index.index_uid).await; } +pub async fn test_metastore_prune_shards< + MetastoreUnderTest: MetastoreService + MetastoreServiceExt + DefaultForTest + ReadWriteShardsForTest, +>() { + let mut metastore = MetastoreUnderTest::default_for_test().await; + + let test_index = TestIndex::create_index_with_source( + &mut metastore, + "test-prune-shards", + SourceConfig::ingest_v2(), + ) + .await; + + let now_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + let oldest_shard_age = 10000u32; + + // Create shards with timestamp intervals of 100s starting from + // now_timestamp - oldest_shard_age + let shards = (0..100) + .map(|shard_id| Shard { + index_uid: Some(test_index.index_uid.clone()), + source_id: test_index.source_id.clone(), + shard_id: Some(ShardId::from(shard_id)), + shard_state: ShardState::Closed as i32, + doc_mapping_uid: Some(DocMappingUid::default()), + publish_position_inclusive: Some(Position::Beginning), + update_timestamp: now_timestamp - oldest_shard_age as i64 + shard_id as i64 * 100, + ..Default::default() + }) + .collect_vec(); + + metastore + .insert_shards(&test_index.index_uid, &test_index.source_id, shards) + .await; + + // noop prune request + { + let prune_index_request = PruneShardsRequest { + index_uid: Some(test_index.index_uid.clone()), + source_id: test_index.source_id.clone(), + max_age: None, + max_count: None, + }; + let response = metastore.prune_shards(prune_index_request).await.unwrap(); + assert_eq!(response.index_uid(), &test_index.index_uid); + assert_eq!(response.source_id, test_index.source_id); + let all_shards = metastore + .list_all_shards(&test_index.index_uid, &test_index.source_id) + .await; + assert_eq!(all_shards.len(), 100); + } + + // delete shards 4 last shards with age limit + { + let prune_index_request = PruneShardsRequest { + index_uid: Some(test_index.index_uid.clone()), + source_id: test_index.source_id.clone(), + max_age: Some(oldest_shard_age - 350), + max_count: None, + }; + let response = metastore.prune_shards(prune_index_request).await.unwrap(); + assert_eq!(response.index_uid(), &test_index.index_uid); + assert_eq!(response.source_id, test_index.source_id); + let mut all_shards = metastore + .list_all_shards(&test_index.index_uid, &test_index.source_id) + .await; + assert_eq!(all_shards.len(), 96); + all_shards.sort_unstable_by_key(|shard| shard.update_timestamp); + assert_eq!(all_shards[0].shard_id(), ShardId::from(4)); + assert_eq!(all_shards[95].shard_id(), ShardId::from(99)); + } + + // delete 6 more shards with count limit + { + let prune_index_request = PruneShardsRequest { + index_uid: Some(test_index.index_uid.clone()), + source_id: test_index.source_id.clone(), + max_age: None, + max_count: Some(90), + }; + let response = metastore.prune_shards(prune_index_request).await.unwrap(); + assert_eq!(response.index_uid(), &test_index.index_uid); + assert_eq!(response.source_id, test_index.source_id); + let mut all_shards = metastore + .list_all_shards(&test_index.index_uid, &test_index.source_id) + .await; + assert_eq!(all_shards.len(), 90); + all_shards.sort_unstable_by_key(|shard| shard.update_timestamp); + assert_eq!(all_shards[0].shard_id(), ShardId::from(10)); + assert_eq!(all_shards[89].shard_id(), ShardId::from(99)); + } + + // age limit is the limiting factor, delete 10 more shards + let prune_index_request = PruneShardsRequest { + index_uid: Some(test_index.index_uid.clone()), + source_id: test_index.source_id.clone(), + max_age: Some(oldest_shard_age - 2950), + max_count: Some(80), + }; + let response = metastore.prune_shards(prune_index_request).await.unwrap(); + assert_eq!(response.index_uid(), &test_index.index_uid); + assert_eq!(response.source_id, test_index.source_id); + let all_shards = metastore + .list_all_shards(&test_index.index_uid, &test_index.source_id) + .await; + assert_eq!(all_shards.len(), 70); + + // count limit is the limiting factor, delete 20 more shards + let prune_index_request = PruneShardsRequest { + index_uid: Some(test_index.index_uid.clone()), + source_id: test_index.source_id.clone(), + max_age: Some(oldest_shard_age - 4000), + max_count: Some(50), + }; + let response = metastore.prune_shards(prune_index_request).await.unwrap(); + assert_eq!(response.index_uid(), &test_index.index_uid); + assert_eq!(response.source_id, test_index.source_id); + let all_shards = metastore + .list_all_shards(&test_index.index_uid, &test_index.source_id) + .await; + assert_eq!(all_shards.len(), 50); + + cleanup_index(&mut metastore, test_index.index_uid).await; +} + pub async fn test_metastore_apply_checkpoint_delta_v2_single_shard< MetastoreUnderTest: MetastoreService + MetastoreServiceExt + DefaultForTest + ReadWriteShardsForTest, >() { diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index b3cd3a7898d..4ce08680e92 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -177,6 +177,9 @@ service MetastoreService { // If the shard did not exist to begin with, the operation is successful and does not return any error. rpc DeleteShards(DeleteShardsRequest) returns (DeleteShardsResponse); + // Deletes outdated shards. This RPC deletes the shards from the metastore. + rpc PruneShards(PruneShardsRequest) returns (PruneShardsResponse); + rpc ListShards(ListShardsRequest) returns (ListShardsResponse); // Index Template API @@ -451,6 +454,20 @@ message DeleteShardsResponse { repeated quickwit.ingest.ShardId failures = 4; } +message PruneShardsRequest { + quickwit.common.IndexUid index_uid = 1; + string source_id = 2; + // The maximum age of the shards to keep, in seconds. + optional uint32 max_age = 5; + // The maximum number of the shards to keep. Delete older shards first. + optional uint32 max_count = 6; +} + +message PruneShardsResponse { + quickwit.common.IndexUid index_uid = 1; + string source_id = 2; +} + message ListShardsRequest { repeated ListShardsSubrequest subrequests = 1; } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 1f0f36db21c..9d67734b564 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -411,6 +411,30 @@ pub struct DeleteShardsResponse { #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PruneShardsRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + /// The maximum age of the shards to keep, in seconds. + #[prost(uint32, optional, tag = "5")] + pub max_age: ::core::option::Option, + /// The maximum number of the shards to keep. Delete older shards first. + #[prost(uint32, optional, tag = "6")] + pub max_count: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PruneShardsResponse { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct ListShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, @@ -732,6 +756,11 @@ impl RpcName for DeleteShardsRequest { "delete_shards" } } +impl RpcName for PruneShardsRequest { + fn rpc_name() -> &'static str { + "prune_shards" + } +} impl RpcName for ListShardsRequest { fn rpc_name() -> &'static str { "list_shards" @@ -900,6 +929,11 @@ pub trait MetastoreService: std::fmt::Debug + Send + Sync + 'static { &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult; + /// Deletes outdated shards. This RPC deletes the shards from the metastore. + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> crate::metastore::MetastoreResult; async fn list_shards( &self, request: ListShardsRequest, @@ -1161,6 +1195,12 @@ impl MetastoreService for MetastoreServiceClient { ) -> crate::metastore::MetastoreResult { self.inner.0.delete_shards(request).await } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.prune_shards(request).await + } async fn list_shards( &self, request: ListShardsRequest, @@ -1355,6 +1395,12 @@ pub mod mock_metastore_service { ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_shards(request).await } + async fn prune_shards( + &self, + request: super::PruneShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.prune_shards(request).await + } async fn list_shards( &self, request: super::ListShardsRequest, @@ -1770,6 +1816,22 @@ impl tower::Service for InnerMetastoreServiceClient { Box::pin(fut) } } +impl tower::Service for InnerMetastoreServiceClient { + type Response = PruneShardsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: PruneShardsRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.prune_shards(request).await }; + Box::pin(fut) + } +} impl tower::Service for InnerMetastoreServiceClient { type Response = ListShardsResponse; type Error = crate::metastore::MetastoreError; @@ -1986,6 +2048,11 @@ struct MetastoreServiceTowerServiceStack { DeleteShardsResponse, crate::metastore::MetastoreError, >, + prune_shards_svc: quickwit_common::tower::BoxService< + PruneShardsRequest, + PruneShardsResponse, + crate::metastore::MetastoreError, + >, list_shards_svc: quickwit_common::tower::BoxService< ListShardsRequest, ListShardsResponse, @@ -2157,6 +2224,12 @@ impl MetastoreService for MetastoreServiceTowerServiceStack { ) -> crate::metastore::MetastoreResult { self.delete_shards_svc.clone().ready().await?.call(request).await } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.prune_shards_svc.clone().ready().await?.call(request).await + } async fn list_shards( &self, request: ListShardsRequest, @@ -2430,6 +2503,16 @@ type DeleteShardsLayer = quickwit_common::tower::BoxLayer< DeleteShardsResponse, crate::metastore::MetastoreError, >; +type PruneShardsLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + PruneShardsRequest, + PruneShardsResponse, + crate::metastore::MetastoreError, + >, + PruneShardsRequest, + PruneShardsResponse, + crate::metastore::MetastoreError, +>; type ListShardsLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< ListShardsRequest, @@ -2515,6 +2598,7 @@ pub struct MetastoreServiceTowerLayerStack { open_shards_layers: Vec, acquire_shards_layers: Vec, delete_shards_layers: Vec, + prune_shards_layers: Vec, list_shards_layers: Vec, create_index_template_layers: Vec, get_index_template_layers: Vec, @@ -3108,6 +3192,31 @@ impl MetastoreServiceTowerLayerStack { crate::metastore::MetastoreError, >, >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + PruneShardsRequest, + PruneShardsResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + PruneShardsRequest, + Response = PruneShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, L: tower::Layer< quickwit_common::tower::BoxService< ListShardsRequest, @@ -3313,6 +3422,8 @@ impl MetastoreServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.delete_shards_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.prune_shards_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.list_shards_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.create_index_template_layers @@ -3778,6 +3889,25 @@ impl MetastoreServiceTowerLayerStack { self.delete_shards_layers.push(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn stack_prune_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + PruneShardsRequest, + PruneShardsResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + PruneShardsRequest, + Response = PruneShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.prune_shards_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn stack_list_shards_layer(mut self, layer: L) -> Self where L: tower::Layer< @@ -4145,6 +4275,14 @@ impl MetastoreServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); + let prune_shards_svc = self + .prune_shards_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); let list_shards_svc = self .list_shards_layers .into_iter() @@ -4218,6 +4356,7 @@ impl MetastoreServiceTowerLayerStack { open_shards_svc, acquire_shards_svc, delete_shards_svc, + prune_shards_svc, list_shards_svc, create_index_template_svc, get_index_template_svc, @@ -4450,6 +4589,12 @@ where Error = crate::metastore::MetastoreError, Future = BoxFuture, > + + tower::Service< + PruneShardsRequest, + Response = PruneShardsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + tower::Service< ListShardsRequest, Response = ListShardsResponse, @@ -4634,6 +4779,12 @@ where ) -> crate::metastore::MetastoreResult { self.clone().call(request).await } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } async fn list_shards( &self, request: ListShardsRequest, @@ -5047,6 +5198,20 @@ where DeleteShardsRequest::rpc_name(), )) } + async fn prune_shards( + &self, + request: PruneShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .prune_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + PruneShardsRequest::rpc_name(), + )) + } async fn list_shards( &self, request: ListShardsRequest, @@ -5422,6 +5587,17 @@ for MetastoreServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn prune_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .prune_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } async fn list_shards( &self, request: tonic::Request, @@ -6299,6 +6475,34 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } + /// Deletes outdated shards. This RPC deletes the shards from the metastore. + pub async fn prune_shards( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.metastore.MetastoreService/PruneShards", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new("quickwit.metastore.MetastoreService", "PruneShards"), + ); + self.inner.unary(req, path, codec).await + } pub async fn list_shards( &mut self, request: impl tonic::IntoRequest, @@ -6658,6 +6862,14 @@ pub mod metastore_service_grpc_server { tonic::Response, tonic::Status, >; + /// Deletes outdated shards. This RPC deletes the shards from the metastore. + async fn prune_shards( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; async fn list_shards( &self, request: tonic::Request, @@ -7884,6 +8096,52 @@ pub mod metastore_service_grpc_server { }; Box::pin(fut) } + "/quickwit.metastore.MetastoreService/PruneShards" => { + #[allow(non_camel_case_types)] + struct PruneShardsSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for PruneShardsSvc { + type Response = super::PruneShardsResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + (*inner).prune_shards(request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = PruneShardsSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } "/quickwit.metastore.MetastoreService/ListShards" => { #[allow(non_camel_case_types)] struct ListShardsSvc(pub Arc); diff --git a/quickwit/quickwit-proto/src/getters.rs b/quickwit/quickwit-proto/src/getters.rs index 9f8df0289ac..ceda0908f03 100644 --- a/quickwit/quickwit-proto/src/getters.rs +++ b/quickwit/quickwit-proto/src/getters.rs @@ -134,6 +134,8 @@ generate_getters! { ListStaleSplitsRequest, MarkSplitsForDeletionRequest, OpenShardSubrequest, + PruneShardsRequest, + PruneShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, From fc5ef169c874be946dcd56616892cd892b0b5b52 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Mon, 2 Sep 2024 16:29:46 +0200 Subject: [PATCH 24/25] Reaquire shards only if they are stale (#5338) --- .../src/source/queue_sources/coordinator.rs | 117 ++++++++--- .../src/source/queue_sources/shared_state.rs | 195 +++++++++++++----- .../src/source/queue_sources/visibility.rs | 2 +- 3 files changed, 228 insertions(+), 86 deletions(-) diff --git a/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs b/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs index bd00840f657..0f92e27ffa3 100644 --- a/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs +++ b/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs @@ -78,7 +78,7 @@ pub struct QueueCoordinator { shared_state: QueueSharedState, local_state: QueueLocalState, publish_token: String, - visible_settings: VisibilitySettings, + visibility_settings: VisibilitySettings, } impl fmt::Debug for QueueCoordinator { @@ -102,6 +102,9 @@ impl QueueCoordinator { metastore: source_runtime.metastore, source_id: source_runtime.pipeline_id.source_id.clone(), index_uid: source_runtime.pipeline_id.index_uid.clone(), + reacquire_grace_period: Duration::from_secs( + 2 * source_runtime.indexing_setting.commit_timeout_secs as u64, + ), }, local_state: QueueLocalState::default(), pipeline_id: source_runtime.pipeline_id, @@ -113,7 +116,7 @@ impl QueueCoordinator { message_type, publish_lock: PublishLock::default(), publish_token: Ulid::new().to_string(), - visible_settings: VisibilitySettings::from_commit_timeout( + visibility_settings: VisibilitySettings::from_commit_timeout( source_runtime.indexing_setting.commit_timeout_secs, ), } @@ -157,7 +160,7 @@ impl QueueCoordinator { async fn poll_messages(&mut self, ctx: &SourceContext) -> Result<(), ActorExitStatus> { let raw_messages = self .queue_receiver - .receive(1, self.visible_settings.deadline_for_receive) + .receive(1, self.visibility_settings.deadline_for_receive) .await?; let mut format_errors = Vec::new(); @@ -215,7 +218,7 @@ impl QueueCoordinator { self.queue.clone(), message.metadata.ack_id.clone(), message.metadata.initial_deadline, - self.visible_settings.clone(), + self.visibility_settings.clone(), ), content: message, position, @@ -254,7 +257,7 @@ impl QueueCoordinator { .await?; if in_progress_ref.batch_reader.is_eof() { self.local_state - .drop_currently_read(self.visible_settings.deadline_for_last_extension) + .drop_currently_read(self.visibility_settings.deadline_for_last_extension) .await?; self.observable_state.num_messages_processed += 1; } @@ -319,7 +322,7 @@ mod tests { use crate::source::doc_file_reader::file_test_helpers::{generate_dummy_doc_file, DUMMY_DOC}; use crate::source::queue_sources::memory_queue::MemoryQueueForTests; use crate::source::queue_sources::message::PreProcessedPayload; - use crate::source::queue_sources::shared_state::shared_state_for_tests::shared_state_for_tests; + use crate::source::queue_sources::shared_state::shared_state_for_tests::init_state; use crate::source::{SourceActor, BATCH_NUM_BYTES_LIMIT}; fn setup_coordinator( @@ -347,7 +350,7 @@ mod tests { source_type: SourceType::Unspecified, storage_resolver: StorageResolver::for_test(), publish_token: Ulid::new().to_string(), - visible_settings: VisibilitySettings::from_commit_timeout(5), + visibility_settings: VisibilitySettings::from_commit_timeout(5), } } @@ -401,7 +404,7 @@ mod tests { #[tokio::test] async fn test_process_empty_queue() { let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests("test-index", Default::default()); + let shared_state = init_state("test-index", Default::default()); let mut coordinator = setup_coordinator(queue.clone(), shared_state); let batches = process_messages(&mut coordinator, queue, &[]).await; assert_eq!(batches.len(), 0); @@ -410,7 +413,7 @@ mod tests { #[tokio::test] async fn test_process_one_small_message() { let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests("test-index", Default::default()); + let shared_state = init_state("test-index", Default::default()); let mut coordinator = setup_coordinator(queue.clone(), shared_state.clone()); let (dummy_doc_file, _) = generate_dummy_doc_file(false, 10).await; let test_uri = Uri::from_str(dummy_doc_file.path().to_str().unwrap()).unwrap(); @@ -424,7 +427,7 @@ mod tests { #[tokio::test] async fn test_process_one_big_message() { let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests("test-index", Default::default()); + let shared_state = init_state("test-index", Default::default()); let mut coordinator = setup_coordinator(queue.clone(), shared_state); let lines = BATCH_NUM_BYTES_LIMIT as usize / DUMMY_DOC.len() + 1; let (dummy_doc_file, _) = generate_dummy_doc_file(true, lines).await; @@ -437,7 +440,7 @@ mod tests { #[tokio::test] async fn test_process_two_messages_different_compression() { let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests("test-index", Default::default()); + let shared_state = init_state("test-index", Default::default()); let mut coordinator = setup_coordinator(queue.clone(), shared_state); let (dummy_doc_file_1, _) = generate_dummy_doc_file(false, 10).await; let test_uri_1 = Uri::from_str(dummy_doc_file_1.path().to_str().unwrap()).unwrap(); @@ -456,7 +459,7 @@ mod tests { #[tokio::test] async fn test_process_local_duplicate_message() { let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests("test-index", Default::default()); + let shared_state = init_state("test-index", Default::default()); let mut coordinator = setup_coordinator(queue.clone(), shared_state); let (dummy_doc_file, _) = generate_dummy_doc_file(false, 10).await; let test_uri = Uri::from_str(dummy_doc_file.path().to_str().unwrap()).unwrap(); @@ -477,11 +480,15 @@ mod tests { let partition_id = PreProcessedPayload::ObjectUri(test_uri.clone()).partition_id(); let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests( + let shared_state = init_state( "test-index", &[( partition_id.clone(), - ("existing_token".to_string(), Position::eof(file_size)), + ( + "existing_token".to_string(), + Position::eof(file_size), + false, + ), )], ); let mut coordinator = setup_coordinator(queue.clone(), shared_state.clone()); @@ -492,30 +499,82 @@ mod tests { assert!(coordinator.local_state.is_completed(&partition_id)); } + #[tokio::test] + async fn test_process_existing_messages() { + let (dummy_doc_file_1, _) = generate_dummy_doc_file(false, 10).await; + let test_uri_1 = Uri::from_str(dummy_doc_file_1.path().to_str().unwrap()).unwrap(); + let partition_id_1 = PreProcessedPayload::ObjectUri(test_uri_1.clone()).partition_id(); + + let (dummy_doc_file_2, _) = generate_dummy_doc_file(false, 10).await; + let test_uri_2 = Uri::from_str(dummy_doc_file_2.path().to_str().unwrap()).unwrap(); + let partition_id_2 = PreProcessedPayload::ObjectUri(test_uri_2.clone()).partition_id(); + + let (dummy_doc_file_3, _) = generate_dummy_doc_file(false, 10).await; + let test_uri_3 = Uri::from_str(dummy_doc_file_3.path().to_str().unwrap()).unwrap(); + let partition_id_3 = PreProcessedPayload::ObjectUri(test_uri_3.clone()).partition_id(); + + let queue = Arc::new(MemoryQueueForTests::new()); + let shared_state = init_state( + "test-index", + &[ + ( + partition_id_1.clone(), + ("existing_token_1".to_string(), Position::Beginning, true), + ), + ( + partition_id_2.clone(), + ( + "existing_token_2".to_string(), + Position::offset((DUMMY_DOC.len() + 1) * 2), + true, + ), + ), + ( + partition_id_3.clone(), + ( + "existing_token_3".to_string(), + Position::offset((DUMMY_DOC.len() + 1) * 6), + false, // should not be processed because not stale yet + ), + ), + ], + ); + let mut coordinator = setup_coordinator(queue.clone(), shared_state.clone()); + let batches = process_messages( + &mut coordinator, + queue, + &[ + (&test_uri_1, "ack-id-1"), + (&test_uri_2, "ack-id-2"), + (&test_uri_3, "ack-id-3"), + ], + ) + .await; + assert_eq!(batches.len(), 2); + assert_eq!(batches.iter().map(|b| b.docs.len()).sum::(), 18); + assert!(coordinator.local_state.is_awaiting_commit(&partition_id_1)); + assert!(coordinator.local_state.is_awaiting_commit(&partition_id_2)); + } + #[tokio::test] async fn test_process_multiple_coordinator() { let queue = Arc::new(MemoryQueueForTests::new()); - let shared_state = shared_state_for_tests("test-index", Default::default()); - let mut proc_1 = setup_coordinator(queue.clone(), shared_state.clone()); - let mut proc_2 = setup_coordinator(queue.clone(), shared_state.clone()); + let shared_state = init_state("test-index", Default::default()); + let mut coord_1 = setup_coordinator(queue.clone(), shared_state.clone()); + let mut coord_2 = setup_coordinator(queue.clone(), shared_state.clone()); let (dummy_doc_file, _) = generate_dummy_doc_file(false, 10).await; let test_uri = Uri::from_str(dummy_doc_file.path().to_str().unwrap()).unwrap(); let partition_id = PreProcessedPayload::ObjectUri(test_uri.clone()).partition_id(); - let batches_1 = process_messages(&mut proc_1, queue.clone(), &[(&test_uri, "ack1")]).await; - let batches_2 = process_messages(&mut proc_2, queue, &[(&test_uri, "ack2")]).await; + let batches_1 = process_messages(&mut coord_1, queue.clone(), &[(&test_uri, "ack1")]).await; + let batches_2 = process_messages(&mut coord_2, queue, &[(&test_uri, "ack2")]).await; assert_eq!(batches_1.len(), 1); assert_eq!(batches_1[0].docs.len(), 10); - assert!(proc_1.local_state.is_awaiting_commit(&partition_id)); - // proc_2 doesn't know for sure what is happening with the message - // (proc_1 might have crashed), so it just acquires it and takes over - // processing - // - // TODO: this test should fail once we implement the grace - // period before a partition can be re-acquired - assert_eq!(batches_2.len(), 1); - assert_eq!(batches_2[0].docs.len(), 10); - assert!(proc_2.local_state.is_awaiting_commit(&partition_id)); + assert!(coord_1.local_state.is_awaiting_commit(&partition_id)); + // proc_2 learns from shared state that the message is likely still + // being processed and skips it + assert_eq!(batches_2.len(), 0); + assert!(!coord_2.local_state.is_tracked(&partition_id)); } } diff --git a/quickwit/quickwit-indexing/src/source/queue_sources/shared_state.rs b/quickwit/quickwit-indexing/src/source/queue_sources/shared_state.rs index 04be2806282..e8ae1dae8f3 100644 --- a/quickwit/quickwit-indexing/src/source/queue_sources/shared_state.rs +++ b/quickwit/quickwit-indexing/src/source/queue_sources/shared_state.rs @@ -18,6 +18,7 @@ // along with this program. If not, see . use std::collections::BTreeMap; +use std::time::Duration; use anyhow::{bail, Context}; use quickwit_metastore::checkpoint::PartitionId; @@ -26,6 +27,7 @@ use quickwit_proto::metastore::{ OpenShardsRequest, }; use quickwit_proto::types::{DocMappingUid, IndexUid, Position, ShardId}; +use time::OffsetDateTime; use tracing::info; use super::message::PreProcessedMessage; @@ -35,6 +37,9 @@ pub struct QueueSharedState { pub metastore: MetastoreServiceClient, pub index_uid: IndexUid, pub source_id: String, + /// Duration after which the processing of a shard is considered stale and + /// should be reacquired + pub reacquire_grace_period: Duration, } impl QueueSharedState { @@ -78,11 +83,13 @@ impl QueueSharedState { let shard = sub.open_shard(); let position = shard.publish_position_inclusive.clone().unwrap_or_default(); let is_owned = sub.open_shard().publish_token.as_deref() == Some(publish_token); + let update_datetime = OffsetDateTime::from_unix_timestamp(shard.update_timestamp) + .context("Invalid shard update timestamp")?; + let is_stale = + OffsetDateTime::now_utc() - update_datetime > self.reacquire_grace_period; if position.is_eof() || (is_owned && position.is_beginning()) { shards.push((partition_id, position)); - } else if !is_owned { - // TODO: Add logic to only re-acquire shards that have a token that is not - // the local token when they haven't been updated recently + } else if !is_owned && is_stale { info!(previous_token = shard.publish_token, "shard re-acquired"); re_acquired_shards.push(shard.shard_id().clone()); } else if is_owned && !position.is_beginning() { @@ -146,6 +153,7 @@ pub async fn checkpoint_messages( pub mod shared_state_for_tests { use std::sync::{Arc, Mutex}; + use itertools::Itertools; use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ AcquireShardsResponse, MockMetastoreService, OpenShardSubresponse, OpenShardsResponse, @@ -153,9 +161,14 @@ pub mod shared_state_for_tests { use super::*; + /// Creates a metastore that mocks the behavior of the Shard API on the open + /// and acquire methods using a simplified in-memory state. pub(super) fn mock_metastore( - initial_state: &[(PartitionId, (String, Position))], + // Shards (token, position, update_timestamp) in the initial state + initial_state: &[(PartitionId, (String, Position, i64))], + // Times open_shards is expected to be called (None <=> no expectation) open_shard_times: Option, + // Times acquire_shards is expected to be called (None <=> no expectation) acquire_times: Option, ) -> MetastoreServiceClient { let mut mock_metastore = MockMetastoreService::new(); @@ -172,16 +185,22 @@ pub mod shared_state_for_tests { .into_iter() .map(|sub_req| { let partition_id: PartitionId = sub_req.shard_id().to_string().into(); - let (token, position) = inner_state_ref + let req_token = sub_req.publish_token.unwrap(); + let (token, position, update_timestamp) = inner_state_ref .lock() .unwrap() .get(&partition_id) .cloned() - .unwrap_or((sub_req.publish_token.unwrap(), Position::Beginning)); - inner_state_ref - .lock() - .unwrap() - .insert(partition_id, (token.clone(), position.clone())); + .unwrap_or(( + req_token.clone(), + Position::Beginning, + OffsetDateTime::now_utc().unix_timestamp(), + )); + + inner_state_ref.lock().unwrap().insert( + partition_id, + (token.clone(), position.clone(), update_timestamp), + ); OpenShardSubresponse { subrequest_id: sub_req.subrequest_id, open_shard: Some(Shard { @@ -194,7 +213,7 @@ pub mod shared_state_for_tests { doc_mapping_uid: sub_req.doc_mapping_uid, publish_position_inclusive: Some(position), shard_state: ShardState::Open as i32, - update_timestamp: 1724158996, + update_timestamp, }), } }) @@ -204,58 +223,77 @@ pub mod shared_state_for_tests { if let Some(times) = open_shard_times { open_shards_expectation.times(times); } - let acquire_shards_expectation = mock_metastore - .expect_acquire_shards() - // .times(acquire_times) - .returning(move |request| { - let acquired_shards = request - .shard_ids - .into_iter() - .map(|shard_id| { - let partition_id: PartitionId = shard_id.to_string().into(); - let (existing_token, position) = inner_state - .lock() - .unwrap() - .get(&partition_id) - .cloned() - .expect("we should never try to acquire a shard that doesn't exist"); - inner_state.lock().unwrap().insert( - partition_id, - (request.publish_token.clone(), position.clone()), - ); - assert_ne!(existing_token, request.publish_token); - Shard { - shard_id: Some(shard_id), - source_id: "dummy".to_string(), - publish_token: Some(request.publish_token.clone()), - index_uid: None, - follower_id: None, - leader_id: "dummy".to_string(), - doc_mapping_uid: None, - publish_position_inclusive: Some(position), - shard_state: ShardState::Open as i32, - update_timestamp: 1724158996, - } - }) - .collect(); - Ok(AcquireShardsResponse { acquired_shards }) - }); + let acquire_shards_expectation = + mock_metastore + .expect_acquire_shards() + .returning(move |request| { + let acquired_shards = request + .shard_ids + .into_iter() + .map(|shard_id| { + let partition_id: PartitionId = shard_id.to_string().into(); + let (existing_token, position, update_timestamp) = inner_state + .lock() + .unwrap() + .get(&partition_id) + .cloned() + .expect( + "we should never try to acquire a shard that doesn't exist", + ); + inner_state.lock().unwrap().insert( + partition_id, + ( + request.publish_token.clone(), + position.clone(), + update_timestamp, + ), + ); + assert_ne!(existing_token, request.publish_token); + Shard { + shard_id: Some(shard_id), + source_id: "dummy".to_string(), + publish_token: Some(request.publish_token.clone()), + index_uid: None, + follower_id: None, + leader_id: "dummy".to_string(), + doc_mapping_uid: None, + publish_position_inclusive: Some(position), + shard_state: ShardState::Open as i32, + update_timestamp, + } + }) + .collect(); + Ok(AcquireShardsResponse { acquired_shards }) + }); if let Some(times) = acquire_times { acquire_shards_expectation.times(times); } MetastoreServiceClient::from_mock(mock_metastore) } - pub fn shared_state_for_tests( + pub fn init_state( index_id: &str, - initial_state: &[(PartitionId, (String, Position))], + // Shards (token, position, is_stale) in the initial state + initial_state: &[(PartitionId, (String, Position, bool))], ) -> QueueSharedState { let index_uid = IndexUid::new_with_random_ulid(index_id); - let metastore = mock_metastore(initial_state, None, None); + let metastore_state = initial_state + .iter() + .map(|(pid, (token, pos, is_stale))| { + let update_timestamp = if *is_stale { + OffsetDateTime::now_utc().unix_timestamp() - 100 + } else { + OffsetDateTime::now_utc().unix_timestamp() + }; + (pid.clone(), (token.clone(), pos.clone(), update_timestamp)) + }) + .collect_vec(); + let metastore = mock_metastore(&metastore_state, None, None); QueueSharedState { metastore, index_uid, source_id: "test-queue-src".to_string(), + reacquire_grace_period: Duration::from_secs(10), } } } @@ -292,13 +330,21 @@ mod tests { async fn test_acquire_shards_with_completed() { let index_id = "test-sqs-index"; let index_uid = IndexUid::new_with_random_ulid(index_id); - let init_state = &[("p1".into(), ("token2".to_string(), Position::eof(100usize)))]; + let init_state = &[( + "p1".into(), + ( + "token2".to_string(), + Position::eof(100usize), + OffsetDateTime::now_utc().unix_timestamp(), + ), + )]; let metastore = mock_metastore(init_state, Some(1), Some(0)); let shared_state = QueueSharedState { metastore, index_uid, source_id: "test-sqs-source".to_string(), + reacquire_grace_period: Duration::from_secs(10), }; let aquired = shared_state @@ -310,12 +356,45 @@ mod tests { } #[tokio::test] - async fn test_re_acquire_shards() { + async fn test_re_acquire_shards_within_grace_period() { + let index_id = "test-sqs-index"; + let index_uid = IndexUid::new_with_random_ulid(index_id); + let init_state = &[( + "p1".into(), + ( + "token2".to_string(), + Position::offset(100usize), + OffsetDateTime::now_utc().unix_timestamp(), + ), + )]; + let metastore = mock_metastore(init_state, Some(1), Some(0)); + + let shared_state = QueueSharedState { + metastore, + index_uid, + source_id: "test-sqs-source".to_string(), + reacquire_grace_period: Duration::from_secs(10), + }; + + let acquired = shared_state + .acquire_partitions("token1", vec!["p1".into(), "p2".into()]) + .await + .unwrap(); + assert_eq!(acquired.len(), 1); + assert!(acquired.contains(&("p2".into(), Position::Beginning))); + } + + #[tokio::test] + async fn test_re_acquire_shards_after_grace_period() { let index_id = "test-sqs-index"; let index_uid = IndexUid::new_with_random_ulid(index_id); let init_state = &[( "p1".into(), - ("token2".to_string(), Position::offset(100usize)), + ( + "token2".to_string(), + Position::offset(100usize), + OffsetDateTime::now_utc().unix_timestamp() - 100, + ), )]; let metastore = mock_metastore(init_state, Some(1), Some(1)); @@ -323,14 +402,13 @@ mod tests { metastore, index_uid, source_id: "test-sqs-source".to_string(), + reacquire_grace_period: Duration::from_secs(10), }; let aquired = shared_state .acquire_partitions("token1", vec!["p1".into(), "p2".into()]) .await .unwrap(); - // TODO: this test should fail once we implement the grace - // period before a partition can be re-acquired assert!(aquired.contains(&("p1".into(), Position::offset(100usize)))); assert!(aquired.contains(&("p2".into(), Position::Beginning))); } @@ -346,13 +424,18 @@ mod tests { let init_state = &[( completed_partition_id.clone(), - ("token2".to_string(), Position::eof(100usize)), + ( + "token2".to_string(), + Position::eof(100usize), + OffsetDateTime::now_utc().unix_timestamp(), + ), )]; let metastore = mock_metastore(init_state, Some(1), Some(0)); let shared_state = QueueSharedState { metastore, index_uid, source_id: "test-sqs-source".to_string(), + reacquire_grace_period: Duration::from_secs(10), }; let checkpointed_msg = checkpoint_messages(&shared_state, "token1", source_messages) diff --git a/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs b/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs index 340a6c05b95..7230186137f 100644 --- a/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs +++ b/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs @@ -40,7 +40,7 @@ pub(super) struct VisibilitySettings { pub deadline_for_default_extension: Duration, /// Rhe timeout for the visibility extension request pub request_timeout: Duration, - /// an extra margin that is substracted from the expected deadline when + /// an extra margin that is subtracted from the expected deadline when /// asserting whether we are still in time to extend the visibility pub request_margin: Duration, } From 707f22a5e13ea902bc9fffd5738a9cf1d794f13d Mon Sep 17 00:00:00 2001 From: trinity-1686a Date: Tue, 3 Sep 2024 11:14:15 +0200 Subject: [PATCH 25/25] ingestv2 doc validation improvements (#5375) * validate timestamp field * let control-plane decide whether shards should validate documents --- .../src/ingest/ingest_controller.rs | 135 +++++++++++++- .../quickwit-control-plane/src/model/mod.rs | 6 + .../src/default_doc_mapper/default_mapper.rs | 41 ++++- .../src/default_doc_mapper/mapping_tree.rs | 2 +- .../quickwit-doc-mapper/src/doc_mapper.rs | 170 +++++++++++++++++- .../src/ingest_v2/broadcast.rs | 2 + .../quickwit-ingest/src/ingest_v2/idle.rs | 2 + .../quickwit-ingest/src/ingest_v2/ingester.rs | 108 ++++++++++- .../quickwit-ingest/src/ingest_v2/models.rs | 10 ++ .../quickwit-ingest/src/ingest_v2/state.rs | 1 + .../protos/quickwit/ingester.proto | 1 + .../quickwit/quickwit.ingest.ingester.rs | 2 + 12 files changed, 461 insertions(+), 19 deletions(-) diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 022886db938..6cfbdcfc7ac 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -779,6 +779,11 @@ impl IngestController { let index_metadata = model .index_metadata(&source_uid.index_uid) .expect("index should exist"); + let validate_docs = model + .source_metadata(source_uid) + .expect("source should exist") + .transform_config + .is_none(); 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)?; @@ -799,6 +804,7 @@ impl IngestController { subrequest_id: subrequest_id as u32, shard: Some(shard), doc_mapping_json, + validate_docs, }; init_shard_subrequests.push(init_shard_subrequest); } @@ -1324,7 +1330,7 @@ mod tests { let shard = subrequest.shard(); assert_eq!(shard.index_uid(), &index_uid_1_clone); - assert_eq!(shard.source_id, "test-source"); + assert_eq!(shard.source_id, source_id); assert_eq!(shard.leader_id, "test-ingester-2"); let successes = vec![InitShardSuccess { @@ -1507,7 +1513,7 @@ mod tests { let shard = subrequest.shard(); assert_eq!(shard.index_uid(), &index_uid_0); - assert_eq!(shard.source_id, "test-source"); + assert_eq!(shard.source_id, source_id); assert_eq!(shard.leader_id, "test-ingester-1"); let successes = vec![InitShardSuccess { @@ -1911,6 +1917,7 @@ mod tests { ..Default::default() }), doc_mapping_json: "{}".to_string(), + validate_docs: false, }, InitShardSubrequest { subrequest_id: 1, @@ -1923,6 +1930,7 @@ mod tests { ..Default::default() }), doc_mapping_json: "{}".to_string(), + validate_docs: false, }, InitShardSubrequest { subrequest_id: 2, @@ -1935,6 +1943,7 @@ mod tests { ..Default::default() }), doc_mapping_json: "{}".to_string(), + validate_docs: false, }, InitShardSubrequest { subrequest_id: 3, @@ -1947,6 +1956,7 @@ mod tests { ..Default::default() }), doc_mapping_json: "{}".to_string(), + validate_docs: false, }, InitShardSubrequest { subrequest_id: 4, @@ -1959,6 +1969,7 @@ mod tests { ..Default::default() }), doc_mapping_json: "{}".to_string(), + validate_docs: false, }, ]; let init_shards_response = controller @@ -2033,6 +2044,10 @@ mod tests { source_id: source_id.clone(), }; let mut index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); + index_metadata.sources.insert( + source_id.clone(), + SourceConfig::for_test(&source_id, quickwit_config::SourceParams::void()), + ); let doc_mapping_json = format!( r#"{{ @@ -2159,8 +2174,12 @@ mod tests { ); let index_uid = IndexUid::for_test("test-index", 0); - let index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); + let mut index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); let source_id: SourceId = "test-source".to_string(); + index_metadata.sources.insert( + source_id.clone(), + SourceConfig::for_test(&source_id, quickwit_config::SourceParams::void()), + ); let source_uid = SourceUid { index_uid: index_uid.clone(), @@ -2228,6 +2247,7 @@ mod tests { assert_eq!(init_shard_request.subrequests.len(), 1); let init_shard_subrequest: &InitShardSubrequest = &init_shard_request.subrequests[0]; + assert!(init_shard_subrequest.validate_docs); Ok(InitShardsResponse { successes: vec![InitShardSuccess { subrequest_id: init_shard_subrequest.subrequest_id, @@ -2311,6 +2331,115 @@ mod tests { .unwrap(); } + #[tokio::test] + async fn test_ingest_controller_disable_validation_when_vrl() { + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_open_shards() + .once() + .returning(|request| { + let subrequest: &OpenShardSubrequest = &request.subrequests[0]; + let shard = Shard { + index_uid: subrequest.index_uid.clone(), + source_id: subrequest.source_id.clone(), + shard_id: subrequest.shard_id.clone(), + shard_state: ShardState::Open as i32, + leader_id: subrequest.leader_id.clone(), + follower_id: subrequest.follower_id.clone(), + doc_mapping_uid: subrequest.doc_mapping_uid, + publish_position_inclusive: Some(Position::Beginning), + publish_token: None, + update_timestamp: 1724158996, + }; + let response = OpenShardsResponse { + subresponses: vec![OpenShardSubresponse { + subrequest_id: subrequest.subrequest_id, + open_shard: Some(shard), + }], + }; + 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, + TEST_SHARD_THROUGHPUT_LIMIT_MIB, + ); + + let index_uid = IndexUid::for_test("test-index", 0); + let mut index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); + let source_id: SourceId = "test-source".to_string(); + let mut source_config = + SourceConfig::for_test(&source_id, quickwit_config::SourceParams::void()); + // set a vrl script + source_config.transform_config = + Some(quickwit_config::TransformConfig::new("".to_string(), None)); + index_metadata + .sources + .insert(source_id.clone(), source_config); + + let source_uid = SourceUid { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + }; + let mut model = ControlPlaneModel::default(); + model.add_index(index_metadata); + let progress = Progress::default(); + + let shards = vec![Shard { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + shard_id: Some(ShardId::from(1)), + leader_id: "test-ingester".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }]; + model.insert_shards(&index_uid, &source_id, shards); + + let mut mock_ingester = MockIngesterService::new(); + + mock_ingester.expect_init_shards().returning( + move |init_shard_request: InitShardsRequest| { + assert_eq!(init_shard_request.subrequests.len(), 1); + let init_shard_subrequest: &InitShardSubrequest = + &init_shard_request.subrequests[0]; + // we have vrl, so no validation + assert!(!init_shard_subrequest.validate_docs); + Ok(InitShardsResponse { + successes: vec![InitShardSuccess { + subrequest_id: init_shard_subrequest.subrequest_id, + shard: init_shard_subrequest.shard.clone(), + }], + failures: Vec::new(), + }) + }, + ); + + let ingester = IngesterServiceClient::from_mock(mock_ingester); + ingester_pool.insert("test-ingester".into(), ingester); + + let shard_infos = BTreeSet::from_iter([ShardInfo { + shard_id: ShardId::from(1), + shard_state: ShardState::Open, + short_term_ingestion_rate: RateMibPerSec(4), + long_term_ingestion_rate: RateMibPerSec(4), + }]); + let local_shards_update = LocalShardsUpdate { + leader_id: "test-ingester".into(), + source_uid: source_uid.clone(), + shard_infos, + }; + + controller + .handle_local_shards_update(local_shards_update, &mut model, &progress) + .await + .unwrap(); + } + #[tokio::test] async fn test_ingest_controller_try_scale_up_shards() { let mut mock_metastore = MockMetastoreService::new(); diff --git a/quickwit/quickwit-control-plane/src/model/mod.rs b/quickwit/quickwit-control-plane/src/model/mod.rs index 4883a9bed25..b378931c5eb 100644 --- a/quickwit/quickwit-control-plane/src/model/mod.rs +++ b/quickwit/quickwit-control-plane/src/model/mod.rs @@ -160,6 +160,12 @@ impl ControlPlaneModel { self.index_table.get(index_uid) } + pub fn source_metadata(&self, source_uid: &SourceUid) -> Option<&SourceConfig> { + self.index_metadata(&source_uid.index_uid)? + .sources + .get(&source_uid.source_id) + } + fn update_metrics(&self) { crate::metrics::CONTROL_PLANE_METRICS .indexes_total diff --git a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs index 512a35e1ab7..a6930b632f3 100644 --- a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs +++ b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs @@ -40,8 +40,8 @@ use tantivy::TantivyDocument as Document; use super::field_mapping_entry::RAW_TOKENIZER_NAME; use super::DefaultDocMapperBuilder; use crate::default_doc_mapper::mapping_tree::{ - build_mapping_tree, map_primitive_json_to_tantivy, JsonValueIterator, MappingNode, - MappingNodeRoot, + build_field_path_from_str, build_mapping_tree, map_primitive_json_to_tantivy, + JsonValueIterator, MappingNode, MappingNodeRoot, }; use crate::default_doc_mapper::FieldMappingType; use crate::doc_mapper::{JsonObject, Partition}; @@ -81,6 +81,8 @@ pub struct DefaultDocMapper { default_search_field_names: Vec, /// Timestamp field name. timestamp_field_name: Option, + /// Timestamp field path (name parsed) + timestamp_field_path: Option>, /// Root node of the field mapping tree. /// See [`MappingNode`]. field_mappings: MappingNode, @@ -197,8 +199,12 @@ impl TryFrom for DefaultDocMapper { if !concatenate_dynamic_fields.is_empty() && dynamic_field.is_none() { bail!("concatenate field has `include_dynamic_fields` set, but index isn't dynamic"); } - if let Some(timestamp_field_path) = &doc_mapping.timestamp_field { - validate_timestamp_field(timestamp_field_path, &field_mappings)?; + let timestamp_field_path = if let Some(timestamp_field_name) = &doc_mapping.timestamp_field + { + validate_timestamp_field(timestamp_field_name, &field_mappings)?; + Some(build_field_path_from_str(timestamp_field_name)) + } else { + None }; let schema = schema_builder.build(); @@ -288,6 +294,7 @@ impl TryFrom for DefaultDocMapper { document_size_field, default_search_field_names, timestamp_field_name: doc_mapping.timestamp_field, + timestamp_field_path, field_mappings, concatenate_dynamic_fields, tag_field_names, @@ -513,6 +520,32 @@ impl DocMapper for DefaultDocMapper { let mut field_path = Vec::new(); self.field_mappings .validate_from_json(json_obj, is_strict, &mut field_path)?; + if let Some(timestamp_field_path) = &self.timestamp_field_path { + let missing_ts_field = + || DocParsingError::RequiredField("timestamp field is required".to_string()); + match ×tamp_field_path[..] { + [] => (), // ? + [single_part] => { + let obj = json_obj.get(single_part).ok_or_else(missing_ts_field)?; + if !(obj.is_string() || obj.is_number()) { + return Err(missing_ts_field()); + } + } + [first_part, more_part @ ..] => { + let mut obj = json_obj.get(first_part).ok_or_else(missing_ts_field)?; + for part in more_part { + obj = obj + .as_object() + .ok_or_else(missing_ts_field)? + .get(part) + .ok_or_else(missing_ts_field)?; + } + if !(obj.is_string() || obj.is_number()) { + return Err(missing_ts_field()); + } + } + }; + } Ok(()) } diff --git a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs index 4439268eb3c..7c82ae0ff95 100644 --- a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs +++ b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs @@ -1410,7 +1410,7 @@ fn field_name_for_field_path(field_path: &[&str]) -> String { /// starting from the root of the document. /// Dots '.' define the boundaries between field names. /// If a dot is part of a field name, it must be escaped with '\'. -fn build_field_path_from_str(field_path_as_str: &str) -> Vec { +pub(crate) fn build_field_path_from_str(field_path_as_str: &str) -> Vec { let mut field_path = Vec::new(); let mut current_path_fragment = String::new(); let mut escaped = false; diff --git a/quickwit/quickwit-doc-mapper/src/doc_mapper.rs b/quickwit/quickwit-doc-mapper/src/doc_mapper.rs index b58d3edaa7c..e8c2704f41e 100644 --- a/quickwit/quickwit-doc-mapper/src/doc_mapper.rs +++ b/quickwit/quickwit-doc-mapper/src/doc_mapper.rs @@ -463,35 +463,187 @@ mod tests { }"#; let doc_mapper = serde_json::from_str::(JSON_CONFIG_VALUE).unwrap(); { - assert!(test_validate_doc_aux(&doc_mapper, r#"{ "body": "toto"}"#).is_ok()); + assert!(test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .is_ok()); } { assert!(matches!( - test_validate_doc_aux(&doc_mapper, r#"{ "response_time": "toto"}"#).unwrap_err(), + test_validate_doc_aux( + &doc_mapper, + r#"{ "response_time": "toto", "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .unwrap_err(), DocParsingError::ValueError(_, _) )); } { - assert!(test_validate_doc_aux(&doc_mapper, r#"{ "response_time": "2.3"}"#).is_ok(),); + assert!(test_validate_doc_aux( + &doc_mapper, + r#"{ "response_time": "2.3", "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .is_ok(),); } { // coercion disabled assert!(matches!( - test_validate_doc_aux(&doc_mapper, r#"{"response_time_no_coercion": "2.3"}"#) - .unwrap_err(), + test_validate_doc_aux( + &doc_mapper, + r#"{"response_time_no_coercion": "2.3", "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .unwrap_err(), DocParsingError::ValueError(_, _) )); } { assert!(matches!( - test_validate_doc_aux(&doc_mapper, r#"{"response_time": [2.3]}"#).unwrap_err(), + test_validate_doc_aux( + &doc_mapper, + r#"{"response_time": [2.3], "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .unwrap_err(), DocParsingError::MultiValuesNotSupported(_) )); } { - assert!( - test_validate_doc_aux(&doc_mapper, r#"{"attributes": {"numbers": [-2]}}"#).is_ok() - ); + assert!(test_validate_doc_aux( + &doc_mapper, + r#"{"attributes": {"numbers": [-2]}, "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .is_ok()); + } + } + + #[test] + fn test_validate_doc_timestamp() { + const JSON_CONFIG_TS_AT_ROOT: &str = r#"{ + "timestamp_field": "timestamp", + "field_mappings": [ + { + "name": "timestamp", + "type": "datetime", + "fast": true + }, + { + "name": "body", + "type": "text" + } + ] + }"#; + const JSON_CONFIG_TS_WITH_DOT: &str = r#"{ + "timestamp_field": "timestamp\\.now", + "field_mappings": [ + { + "name": "timestamp.now", + "type": "datetime", + "fast": true + }, + { + "name": "body", + "type": "text" + } + ] + }"#; + const JSON_CONFIG_TS_NESTED: &str = r#"{ + "timestamp_field": "doc.timestamp", + "field_mappings": [ + { + "name": "doc", + "type": "object", + "field_mappings": [ + { + "name": "timestamp", + "type": "datetime", + "fast": true + } + ] + }, + { + "name": "body", + "type": "text" + } + ] + }"#; + let doc_mapper = serde_json::from_str::(JSON_CONFIG_TS_AT_ROOT).unwrap(); + { + assert!(test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "timestamp": "2024-01-01T01:01:01Z"}"# + ) + .is_ok()); + } + { + assert!(matches!( + test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "timestamp": "invalid timestamp"}"# + ) + .unwrap_err(), + DocParsingError::ValueError(_, _), + )); + } + { + assert!(matches!( + test_validate_doc_aux(&doc_mapper, r#"{ "body": "toto", "timestamp": null}"#) + .unwrap_err(), + DocParsingError::RequiredField(_), + )); + } + { + assert!(matches!( + test_validate_doc_aux(&doc_mapper, r#"{ "body": "toto"}"#).unwrap_err(), + DocParsingError::RequiredField(_), + )); + } + + let doc_mapper = serde_json::from_str::(JSON_CONFIG_TS_WITH_DOT).unwrap(); + { + assert!(test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "timestamp.now": "2024-01-01T01:01:01Z"}"# + ) + .is_ok()); + } + { + assert!(matches!( + test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "timestamp.now": "invalid timestamp"}"# + ) + .unwrap_err(), + DocParsingError::ValueError(_, _), + )); + } + { + assert!(matches!( + test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "timestamp": {"now": "2024-01-01T01:01:01Z"}}"# + ) + .unwrap_err(), + DocParsingError::RequiredField(_), + )); + } + + let doc_mapper = serde_json::from_str::(JSON_CONFIG_TS_NESTED).unwrap(); + { + assert!(test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "doc":{"timestamp": "2024-01-01T01:01:01Z"}}"# + ) + .is_ok()); + } + { + assert!(matches!( + test_validate_doc_aux( + &doc_mapper, + r#"{ "body": "toto", "doc.timestamp": "2024-01-01T01:01:01Z"}"# + ) + .unwrap_err(), + DocParsingError::RequiredField(_), + )); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs index 565598ae2b9..1d2a639b66c 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs @@ -606,6 +606,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); state_guard.shards.insert(queue_id_00.clone(), shard_00); @@ -616,6 +617,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); shard_01.is_advertisable = true; state_guard.shards.insert(queue_id_01.clone(), shard_01); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs index 0263e23d194..a94a6168598 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs @@ -107,6 +107,7 @@ mod tests { Position::Beginning, None, now - idle_shard_timeout, + false, ); let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); state_guard.shards.insert(queue_id_01.clone(), shard_01); @@ -117,6 +118,7 @@ mod tests { Position::Beginning, None, now - idle_shard_timeout / 2, + false, ); let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); state_guard.shards.insert(queue_id_02.clone(), shard_02); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index fb9a9980c8b..7412a25cf28 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -192,6 +192,7 @@ impl Ingester { shard: Shard, doc_mapping_json: &str, now: Instant, + validate: bool, ) -> IngestV2Result<()> { let queue_id = shard.queue_id(); info!( @@ -252,6 +253,7 @@ impl Ingester { Position::Beginning, doc_mapper, now, + validate, ) } else { IngesterShard::new_solo( @@ -260,6 +262,7 @@ impl Ingester { Position::Beginning, Some(doc_mapper), now, + validate, ) }; entry.insert(primary_shard); @@ -516,6 +519,7 @@ impl Ingester { continue; } let doc_mapper = shard.doc_mapper_opt.clone().expect("shard should be open"); + let validate_shard = shard.validate; let follower_id_opt = shard.follower_id_opt().cloned(); let from_position_exclusive = shard.replication_position_inclusive.clone(); @@ -570,8 +574,12 @@ impl Ingester { // Total number of bytes (valid and invalid documents) let original_batch_num_bytes = doc_batch.num_bytes() as u64; - let (valid_doc_batch, parse_failures) = - validate_doc_batch(doc_batch, doc_mapper).await?; + + let (valid_doc_batch, parse_failures) = if validate_shard { + validate_doc_batch(doc_batch, doc_mapper).await? + } else { + (doc_batch, Vec::new()) + }; if valid_doc_batch.is_empty() { crate::metrics::INGEST_METRICS @@ -947,6 +955,7 @@ impl Ingester { subrequest.shard().clone(), &subrequest.doc_mapping_json, now, + subrequest.validate_docs, ) .await; if init_primary_shard_result.is_ok() { @@ -1551,6 +1560,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); state_guard.shards.insert(queue_id_00.clone(), shard_00); @@ -1561,6 +1571,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); shard_01.is_advertisable = true; state_guard.shards.insert(queue_id_01.clone(), shard_01); @@ -1652,6 +1663,7 @@ mod tests { primary_shard, &doc_mapping_json, Instant::now(), + true, ) .await .unwrap(); @@ -1694,6 +1706,7 @@ mod tests { subrequest_id: 0, shard: Some(shard.clone()), doc_mapping_json, + validate_docs: true, }], }; let response = ingester.init_shards(init_shards_request).await.unwrap(); @@ -1744,6 +1757,7 @@ mod tests { ..Default::default() }), doc_mapping_json: doc_mapping_json.clone(), + validate_docs: true, }, InitShardSubrequest { subrequest_id: 1, @@ -1757,6 +1771,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }, ], }; @@ -1867,6 +1882,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }], }; let response = ingester.init_shards(init_shards_request).await.unwrap(); @@ -1937,6 +1953,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }], }; let response = ingester.init_shards(init_shards_request).await.unwrap(); @@ -1984,6 +2001,66 @@ mod tests { assert!(parse_failure_2.message.contains("not declared")); } + #[tokio::test] + async fn test_ingester_persist_doesnt_validates_docs_when_requested() { + 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(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, + validate_docs: false, + }], + }; + 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([ + "", // invalid + "[]", // invalid + r#"{"foo": "bar"}"#, // invalid + r#"{"doc": "test-doc-000"}"#, // valid + ])), + }], + }; + 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.num_persisted_docs, 4); + assert_eq!(persist_success.parse_failures.len(), 0); + } + #[tokio::test] async fn test_ingester_persist_checks_capacity_before_validating_docs() { let (ingester_ctx, ingester) = IngesterForTest::default() @@ -2014,6 +2091,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }], }; let response = ingester.init_shards(init_shards_request).await.unwrap(); @@ -2074,6 +2152,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }], }; let response = ingester.init_shards(init_shards_request).await.unwrap(); @@ -2125,6 +2204,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); state_guard.shards.insert(queue_id.clone(), solo_shard); @@ -2190,6 +2270,7 @@ mod tests { Position::Beginning, Some(doc_mapper), Instant::now(), + false, ); state_guard.shards.insert(queue_id.clone(), solo_shard); @@ -2276,6 +2357,7 @@ mod tests { ..Default::default() }), doc_mapping_json: doc_mapping_json.clone(), + validate_docs: true, }, InitShardSubrequest { subrequest_id: 1, @@ -2290,6 +2372,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }, ], }; @@ -2481,6 +2564,7 @@ mod tests { ..Default::default() }), doc_mapping_json: doc_mapping_json.clone(), + validate_docs: true, }, InitShardSubrequest { subrequest_id: 1, @@ -2495,6 +2579,7 @@ mod tests { ..Default::default() }), doc_mapping_json, + validate_docs: true, }, ], }; @@ -2618,6 +2703,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); ingester .state @@ -2696,6 +2782,7 @@ mod tests { primary_shard, &doc_mapping_json, Instant::now(), + true, ) .await .unwrap(); @@ -2776,6 +2863,7 @@ mod tests { primary_shard, &doc_mapping_json, Instant::now(), + true, ) .await .unwrap(); @@ -2898,6 +2986,7 @@ mod tests { shard, &doc_mapping_json, Instant::now(), + true, ) .await .unwrap(); @@ -3022,6 +3111,7 @@ mod tests { shard_01, &doc_mapping_json_01, now, + true, ) .await .unwrap(); @@ -3032,6 +3122,7 @@ mod tests { shard_02, &doc_mapping_json_02, now, + true, ) .await .unwrap(); @@ -3122,6 +3213,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); state_guard.shards.insert(queue_id.clone(), solo_shard); @@ -3232,6 +3324,7 @@ mod tests { shard_01, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3242,6 +3335,7 @@ mod tests { shard_02, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3314,6 +3408,7 @@ mod tests { shard_17, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3325,6 +3420,7 @@ mod tests { shard_18, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3382,6 +3478,7 @@ mod tests { shard, &doc_mapping_json, Instant::now(), + true, ) .await .unwrap(); @@ -3495,6 +3592,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ), ); ingester.check_decommissioning_status(&mut state_guard); @@ -3551,6 +3649,7 @@ mod tests { shard_01, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3561,6 +3660,7 @@ mod tests { shard_02, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3656,6 +3756,7 @@ mod tests { shard_01, &doc_mapping_json, now - idle_shard_timeout, + true, ) .await .unwrap(); @@ -3723,6 +3824,7 @@ mod tests { shard_01, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3733,6 +3835,7 @@ mod tests { shard_02, &doc_mapping_json, now, + true, ) .await .unwrap(); @@ -3743,6 +3846,7 @@ mod tests { shard_03, &doc_mapping_json, now, + true, ) .await .unwrap(); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/models.rs b/quickwit/quickwit-ingest/src/ingest_v2/models.rs index 8d93954f9f9..79d2932c9af 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/models.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/models.rs @@ -55,6 +55,9 @@ pub(super) struct IngesterShard { pub is_advertisable: bool, /// Document mapper for the shard. Replica shards and closed solo shards do not have one. pub doc_mapper_opt: Option>, + /// Whether to validate documents in this shard. True if no preprocessing (VRL) will happen + /// before indexing. + pub validate: bool, pub shard_status_tx: watch::Sender, pub shard_status_rx: watch::Receiver, /// Instant at which the shard was last written to. @@ -69,6 +72,7 @@ impl IngesterShard { truncation_position_inclusive: Position, doc_mapper: Arc, now: Instant, + validate: bool, ) -> Self { let shard_status = (shard_state, replication_position_inclusive.clone()); let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); @@ -79,6 +83,7 @@ impl IngesterShard { truncation_position_inclusive, is_advertisable: false, doc_mapper_opt: Some(doc_mapper), + validate, shard_status_tx, shard_status_rx, last_write_instant: now, @@ -103,6 +108,7 @@ impl IngesterShard { // anyway. is_advertisable: false, doc_mapper_opt: None, + validate: false, shard_status_tx, shard_status_rx, last_write_instant: now, @@ -115,6 +121,7 @@ impl IngesterShard { truncation_position_inclusive: Position, doc_mapper_opt: Option>, now: Instant, + validate: bool, ) -> Self { let shard_status = (shard_state, replication_position_inclusive.clone()); let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); @@ -125,6 +132,7 @@ impl IngesterShard { truncation_position_inclusive, is_advertisable: false, doc_mapper_opt, + validate, shard_status_tx, shard_status_rx, last_write_instant: now, @@ -253,6 +261,7 @@ mod tests { Position::Beginning, doc_mapper, Instant::now(), + true, ); assert!(matches!( &primary_shard.shard_type, @@ -305,6 +314,7 @@ mod tests { Position::Beginning, None, Instant::now(), + false, ); solo_shard.assert_is_solo(); assert!(!solo_shard.is_replica()); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/state.rs b/quickwit/quickwit-ingest/src/ingest_v2/state.rs index 050155919d7..11993a6cad4 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/state.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/state.rs @@ -190,6 +190,7 @@ impl IngesterState { truncation_position_inclusive, None, now, + false, ); // We want to advertise the shard as read-only right away. solo_shard.is_advertisable = true; diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index a65d7d9fcfd..8874176b941 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -260,6 +260,7 @@ message InitShardSubrequest { uint32 subrequest_id = 1; quickwit.ingest.Shard shard = 2; string doc_mapping_json = 3; + bool validate_docs = 4; } message InitShardsResponse { 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 0b169811727..ccb13a5e44d 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -360,6 +360,8 @@ pub struct InitShardSubrequest { pub shard: ::core::option::Option, #[prost(string, tag = "3")] pub doc_mapping_json: ::prost::alloc::string::String, + #[prost(bool, tag = "4")] + pub validate_docs: bool, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)]