From 63a2dca6e76091654342173b49127547788be69c Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Wed, 25 Oct 2023 08:11:16 -0400 Subject: [PATCH] Report closed shards and unavailable leaders to control plane (#3934) --- quickwit/Cargo.lock | 361 ++++++----- quickwit/quickwit-cli/src/tool.rs | 2 +- quickwit/quickwit-cluster/src/change.rs | 2 +- quickwit/quickwit-cluster/src/cluster.rs | 2 +- quickwit/quickwit-cluster/src/lib.rs | 2 +- quickwit/quickwit-cluster/src/member.rs | 2 +- .../quickwit-config/src/index_config/mod.rs | 2 +- .../src/control_plane.rs | 4 +- .../src/control_plane_model.rs | 87 ++- .../src/indexing_plan.rs | 2 +- .../src/indexing_scheduler.rs | 2 +- .../src/ingest/ingest_controller.rs | 189 +++++- quickwit/quickwit-control-plane/src/tests.rs | 2 +- .../src/garbage_collection.rs | 4 +- .../quickwit-index-management/src/index.rs | 3 +- quickwit/quickwit-indexing/failpoints/mod.rs | 2 +- .../quickwit-indexing/src/actors/indexer.rs | 4 +- .../src/actors/indexing_pipeline.rs | 2 +- .../src/actors/indexing_service.rs | 2 +- .../src/actors/merge_pipeline.rs | 2 +- .../src/actors/merge_planner.rs | 2 +- .../quickwit-indexing/src/actors/packager.rs | 2 +- .../quickwit-indexing/src/actors/publisher.rs | 2 +- .../quickwit-indexing/src/merge_policy/mod.rs | 2 +- quickwit/quickwit-indexing/src/models/mod.rs | 2 +- .../src/source/file_source.rs | 2 +- .../src/source/gcp_pubsub_source.rs | 2 +- .../src/source/ingest/mod.rs | 2 +- .../src/source/ingest_api_source.rs | 2 +- .../src/source/kafka_source.rs | 4 +- quickwit/quickwit-indexing/src/source/mod.rs | 2 +- .../src/source/source_factory.rs | 2 +- .../src/source/vec_source.rs | 2 +- .../src/source/void_source.rs | 2 +- quickwit/quickwit-indexing/src/test_utils.rs | 2 +- .../quickwit-ingest/src/ingest_v2/ingest.md | 12 + .../quickwit-ingest/src/ingest_v2/ingester.rs | 14 +- quickwit/quickwit-ingest/src/ingest_v2/mod.rs | 2 +- .../src/ingest_v2/replication.rs | 3 + .../quickwit-ingest/src/ingest_v2/router.rs | 295 +++++++-- .../src/ingest_v2/shard_table.rs | 589 +++++++++++++++--- .../quickwit-jaeger/src/integration_tests.rs | 2 +- .../src/actors/delete_task_pipeline.rs | 2 +- .../src/actors/delete_task_planner.rs | 2 +- .../src/actors/delete_task_service.rs | 2 +- .../src/actors/garbage_collector.rs | 2 +- .../src/actors/retention_policy_executor.rs | 2 +- .../src/retention_policy_execution.rs | 2 +- .../file_backed_index/mod.rs | 2 +- .../file_backed_index/serialize.rs | 2 +- .../metastore/file_backed_metastore/mod.rs | 4 +- .../src/metastore/index_metadata/serialize.rs | 2 +- .../quickwit-metastore/src/metastore/mod.rs | 2 +- .../src/metastore/postgresql_metastore.rs | 4 +- .../src/metastore/postgresql_model.rs | 2 +- .../quickwit-metastore/src/split_metadata.rs | 2 +- .../src/split_metadata_version.rs | 2 +- quickwit/quickwit-metastore/src/tests.rs | 2 +- .../protos/quickwit/control_plane.proto | 10 +- .../protos/quickwit/ingester.proto | 4 +- .../protos/quickwit/metastore.proto | 1 - .../protos/quickwit/router.proto | 17 +- .../quickwit/quickwit.control_plane.rs | 17 +- .../quickwit/quickwit.ingest.ingester.rs | 10 +- .../quickwit/quickwit.ingest.router.rs | 21 +- quickwit/quickwit-proto/src/error.rs | 3 + quickwit/quickwit-proto/src/ingest/mod.rs | 8 +- quickwit/quickwit-proto/src/lib.rs | 3 +- .../quickwit-proto/src/metastore/events.rs | 2 +- quickwit/quickwit-proto/src/metastore/mod.rs | 4 +- quickwit/quickwit-proto/src/types/mod.rs | 31 +- quickwit/quickwit-search/src/lib.rs | 2 +- quickwit/quickwit-search/src/root.rs | 2 +- .../quickwit-search/src/scroll_context.rs | 2 +- .../src/delete_task_api/handler.rs | 2 +- .../src/index_api/rest_handler.rs | 2 +- .../src/ingest_api/rest_handler.rs | 2 +- quickwit/quickwit-serve/src/lib.rs | 2 +- 78 files changed, 1356 insertions(+), 452 deletions(-) create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/ingest.md diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index ec25546b0a0..2310258e725 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -46,9 +46,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.7.6" +version = "0.7.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +checksum = "5a824f2aa7e75a0c98c5a504fceb80649e9c35265d44525b5f94de4771a395cd" dependencies = [ "getrandom 0.2.10", "once_cell", @@ -57,21 +57,22 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" +checksum = "91429305e9f0a25f6205c5b8e0d2db09e0708a7a6df0f42212bb56c32c8ac97a" dependencies = [ "cfg-if", "getrandom 0.2.10", "once_cell", "version_check", + "zerocopy", ] [[package]] name = "aho-corasick" -version = "1.1.1" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea5d730647d4fadd988536d06fecce94b7b4f2a7efdae548f1cf4b63205518ab" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" dependencies = [ "memchr", ] @@ -248,9 +249,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb42b2197bf15ccb092b62c74515dbd8b86d0effd934795f6687c93b6e679a2c" +checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" dependencies = [ "flate2", "futures-core", @@ -295,9 +296,9 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.73" +version = "0.1.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", @@ -775,7 +776,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "86b0f0eea648347e40f5f7f7e6bfea4553bcefad0fbf52044ea339e5ce3aba61" dependencies = [ "async-trait", - "base64 0.21.4", + "base64 0.21.5", "bytes", "dyn-clone", "futures", @@ -881,9 +882,9 @@ checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" [[package]] name = "base64" -version = "0.21.4" +version = "0.21.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" [[package]] name = "base64-simd" @@ -933,9 +934,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.4.0" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4682ae6287fcf752ecaabbfcc7b6f9b72aa33933dc23a554d853aea8eea8635" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" dependencies = [ "serde", ] @@ -1064,9 +1065,9 @@ dependencies = [ [[package]] name = "bytecount" -version = "0.6.4" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad152d03a2c813c80bb94fedbf3a3f02b28f793e39e7c214c8a0bcc196343de7" +checksum = "e1e5f035d16fc623ae5f74981db80a439803888314e3a555fd6f04acd51a3205" [[package]] name = "byteorder" @@ -1284,18 +1285,18 @@ dependencies = [ [[package]] name = "clap" -version = "4.4.6" +version = "4.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d04704f56c2cde07f43e8e2c154b43f216dc5c92fc98ada720177362f953b956" +checksum = "ac495e00dcec98c83465d5ad66c5c4fabd652fd6686e7c6269b117e729a6f17b" dependencies = [ "clap_builder", ] [[package]] name = "clap_builder" -version = "4.4.6" +version = "4.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e231faeaca65ebd1ea3c737966bf858971cd38c3849107aa3ea7de90a804e45" +checksum = "c77ed9a32a62e6ca27175d00d29d05ca32e396ea1eb5fb01d8256b669cec7663" dependencies = [ "anstream", "anstyle", @@ -1305,9 +1306,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.5.1" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd7cc57abe963c6d3b9d8be5b06ba7c8957a930305ca90304f24ef040aa6f961" +checksum = "702fc72eb24e5a1e48ce58027a675bc24edd52096d5397d4aea7c6dd9eca0bd1" [[package]] name = "cmake" @@ -1445,9 +1446,9 @@ checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" [[package]] name = "cpufeatures" -version = "0.2.9" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a17b76ff3a4162b0b27f354a0c87015ddad39d35f9c0c36607a3bdd175dde1f1" +checksum = "3fbc60abd742b35f2492f808e1abbb83d45f72db402e14c55057edc9c7b1e9e4" dependencies = [ "libc", ] @@ -1710,7 +1711,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "lock_api", "once_cell", "parking_lot_core", @@ -1766,9 +1767,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2696e8a945f658fd14dc3b87242e6b80cd0f36ff04ea560fa39082368847946" +checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" dependencies = [ "serde", ] @@ -1862,7 +1863,7 @@ checksum = "53ecafc952c4528d9b51a458d1a8904b81783feff9fde08ab6ed2545ff396872" dependencies = [ "cfg-if", "libc", - "socket2 0.4.9", + "socket2 0.4.10", "winapi 0.3.9", ] @@ -2183,9 +2184,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flate2" -version = "1.0.27" +version = "1.0.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6c98ee8095e9d1dcbf2fcc6d95acccb90d1c81db1e44725c6a984b1dbdfb010" +checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" dependencies = [ "crc32fast", "miniz_oxide", @@ -2254,7 +2255,7 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "29f9df8a11882c4e3335eb2d18a0137c505d9ca927470b0cac9c6f0ae07d28f7" dependencies = [ - "rustix 0.38.17", + "rustix 0.38.20", "windows-sys 0.48.0", ] @@ -2451,7 +2452,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bb6624c70caf330298b84a9ad1537ee7a5de788a5b9a06a3bbe206260943011" dependencies = [ "async-trait", - "base64 0.21.4", + "base64 0.21.5", "google-cloud-metadata", "google-cloud-token", "home", @@ -2473,7 +2474,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "931bedb2264cb00f914b0a6a5c304e34865c34306632d3932e0951a073e4a67d" dependencies = [ "async-trait", - "base64 0.21.4", + "base64 0.21.5", "google-cloud-metadata", "google-cloud-token", "home", @@ -2642,7 +2643,7 @@ version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" dependencies = [ - "ahash 0.7.6", + "ahash 0.7.7", ] [[package]] @@ -2651,16 +2652,16 @@ version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "43a3c133739dddd0d2990f9a4bdf8eb4b21ef50e4851ca85ab661199821d510e" dependencies = [ - "ahash 0.8.3", + "ahash 0.8.6", ] [[package]] name = "hashbrown" -version = "0.14.1" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dfda62a12f55daeae5015f81b0baea145391cb4520f86c248fc615d72640d12" +checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" dependencies = [ - "ahash 0.8.3", + "ahash 0.8.6", "allocator-api2", ] @@ -2670,7 +2671,7 @@ version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8094feaf31ff591f651a2664fb9cfd92bba7a60ce3197265e9482ebe753c8f7" dependencies = [ - "hashbrown 0.14.1", + "hashbrown 0.14.2", ] [[package]] @@ -2692,7 +2693,7 @@ version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "06683b93020a07e3dbcf5f8c0f6d40080d725bea7936fc01ad345c01b97dc270" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bytes", "headers-core", "http", @@ -2869,7 +2870,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.9", + "socket2 0.4.10", "tokio", "tower-service", "tracing", @@ -2901,7 +2902,7 @@ dependencies = [ "http", "hyper", "log", - "rustls 0.21.7", + "rustls 0.21.8", "rustls-native-certs", "tokio", "tokio-rustls 0.24.1", @@ -2934,16 +2935,16 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.57" +version = "0.1.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fad5b825842d2b38bd206f3e81d6957625fd7f0a361e345c30e01a0ae2dd613" +checksum = "8326b86b6cff230b97d0d312a6c40a60726df3332e721f72a1b035f451663b20" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "windows", + "windows-core", ] [[package]] @@ -2989,7 +2990,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8adf3ddd720272c6ea8bf59463c04e0f93d0bbf7c5439b691bca2987e0270897" dependencies = [ "equivalent", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "serde", ] @@ -3059,9 +3060,9 @@ dependencies = [ [[package]] name = "ipnet" -version = "2.8.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" +checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" [[package]] name = "ipnetwork" @@ -3079,7 +3080,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cb0889898416213fab133e1d33a0e5858a48177452750691bde3666d0fdbaf8b" dependencies = [ "hermit-abi", - "rustix 0.38.17", + "rustix 0.38.20", "windows-sys 0.48.0", ] @@ -3109,9 +3110,9 @@ checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" [[package]] name = "jobserver" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "936cfd212a0155903bcbc060e316fb6cc7cbf2e1907329391ebadc1fe0ce77c2" +checksum = "8c37f63953c4c63420ed5fd3d6d398c719489b9f872b9fa683262f8edd363c7d" dependencies = [ "libc", ] @@ -3137,7 +3138,7 @@ version = "8.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6971da4d9c3aa03c3d8f3ff0f4155b534aad021292003895a469716b2a230378" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "pem", "ring 0.16.20", "serde", @@ -3488,9 +3489,9 @@ checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] name = "lock_api" -version = "0.4.10" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1cc9717a20b1bb222f333e6a92fd32f7d8a18ddc5a3191a11af45dcbf4dcd16" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" dependencies = [ "autocfg", "scopeguard", @@ -3522,7 +3523,7 @@ version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1efa59af2ddfad1854ae27d75009d538d0998b4b2fd47083e743ac1a10e46c60" dependencies = [ - "hashbrown 0.14.1", + "hashbrown 0.14.2", ] [[package]] @@ -3661,9 +3662,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" +checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" dependencies = [ "libc", "wasi 0.11.0+wasi-snapshot-preview1", @@ -4014,7 +4015,7 @@ version = "0.10.57" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bac25ee399abb46215765b1cb35bc0212377e58a061560d8b29b024fd0430e7c" dependencies = [ - "bitflags 2.4.0", + "bitflags 2.4.1", "cfg-if", "foreign-types", "libc", @@ -4143,18 +4144,18 @@ dependencies = [ [[package]] name = "ordered-float" -version = "2.10.0" +version = "2.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7940cf2ca942593318d07fcf2596cdca60a85c9e7fab408a5e21a4f9dcd40d87" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" dependencies = [ "num-traits", ] [[package]] name = "ordered-float" -version = "3.9.1" +version = "3.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a54938017eacd63036332b4ae5c8a49fc8c0c1d6d629893057e4f13609edd06" +checksum = "f1e1c390732d15f1d48471625cd92d154e66db2c56645e29a9cd26f4699f72dc" dependencies = [ "num-traits", ] @@ -4229,9 +4230,9 @@ dependencies = [ [[package]] name = "parking" -version = "2.1.1" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e52c774a4c39359c1d1c52e43f73dd91a75a614652c825408eec30c95a9b2067" +checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" [[package]] name = "parking_lot" @@ -4245,13 +4246,13 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.8" +version = "0.9.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93f00c865fe7cabf650081affecd3871070f26767e7b2070a3ffae14c654b447" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.3.5", + "redox_syscall 0.4.1", "smallvec", "windows-targets 0.48.5", ] @@ -4566,9 +4567,9 @@ dependencies = [ [[package]] name = "portable-atomic" -version = "1.4.3" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31114a898e107c51bb1609ffaf55a0e011cf6a4d7f1170d0015a165082c0338b" +checksum = "b559898e0b4931ed2d3b959ab0c2da4d99cc644c4b0b1a35b4d344027f474023" [[package]] name = "postcard" @@ -4758,7 +4759,7 @@ checksum = "7c003ac8c77cb07bb74f5f198bce836a689bcd5a42574612bf14d17bfd08c20e" dependencies = [ "bit-set", "bit-vec", - "bitflags 2.4.0", + "bitflags 2.4.1", "lazy_static", "num-traits", "rand 0.8.5", @@ -5217,7 +5218,7 @@ name = "quickwit-doc-mapper" version = "0.6.3" dependencies = [ "anyhow", - "base64 0.21.4", + "base64 0.21.5", "criterion", "dyn-clone", "fnv", @@ -5418,7 +5419,7 @@ version = "0.6.3" dependencies = [ "anyhow", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "itertools 0.11.0", "once_cell", "prost", @@ -5547,7 +5548,7 @@ version = "0.6.3" dependencies = [ "anyhow", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "once_cell", "prost", "quickwit-actors", @@ -5604,7 +5605,7 @@ name = "quickwit-query" version = "0.6.3" dependencies = [ "anyhow", - "base64 0.21.4", + "base64 0.21.5", "criterion", "fnv", "hex", @@ -5656,7 +5657,7 @@ dependencies = [ "anyhow", "assert-json-diff 2.0.2", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "bytes", "chitchat", "fnv", @@ -5777,7 +5778,7 @@ dependencies = [ "azure_core", "azure_storage", "azure_storage_blobs", - "base64 0.21.4", + "base64 0.21.5", "byte-unit", "bytes", "fnv", @@ -6008,6 +6009,15 @@ dependencies = [ "bitflags 1.3.2", ] +[[package]] +name = "redox_syscall" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +dependencies = [ + "bitflags 1.3.2", +] + [[package]] name = "redox_users" version = "0.4.3" @@ -6021,14 +6031,14 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.0" +version = "1.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d119d7c7ca818f8a53c300863d4f87566aac09943aef5b355bb83969dae75d87" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.0", - "regex-syntax 0.8.0", + "regex-automata 0.4.3", + "regex-syntax 0.8.2", ] [[package]] @@ -6042,13 +6052,13 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.0" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d58da636bd923eae52b7e9120271cbefb16f399069ee566ca5ebf9c30e32238" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.8.0", + "regex-syntax 0.8.2", ] [[package]] @@ -6065,9 +6075,9 @@ checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" [[package]] name = "regex-syntax" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3cbb081b9784b07cceb8824c8583f86db4814d172ab043f3c23f7dc600bf83d" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" [[package]] name = "rend" @@ -6084,7 +6094,7 @@ version = "0.11.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bytes", "encoding_rs", "futures-core", @@ -6103,7 +6113,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.7", + "rustls 0.21.8", "rustls-pemfile", "serde", "serde_json", @@ -6146,9 +6156,9 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.2" +version = "0.17.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "911b295d2d302948838c8ac142da1ee09fa7863163b44e6715bc9357905878b8" +checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" dependencies = [ "cc", "getrandom 0.2.10", @@ -6323,11 +6333,11 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.17" +version = "0.38.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f25469e9ae0f3d0047ca8b93fc56843f38e6774f0914a107ff8b41be8be8e0b7" +checksum = "67ce50cb2e16c2903e30d1cbccfd8387a74b9d4c938b6a4c5ec6cc7556f7a8a0" dependencies = [ - "bitflags 2.4.0", + "bitflags 2.4.1", "errno", "libc", "linux-raw-sys 0.4.10", @@ -6348,13 +6358,13 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.7" +version = "0.21.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd8d6c9f025a446bc4d18ad9632e69aec8f287aa84499ee335599fabd20c3fd8" +checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" dependencies = [ "log", - "ring 0.16.20", - "rustls-webpki 0.101.6", + "ring 0.17.5", + "rustls-webpki 0.101.7", "sct", ] @@ -6376,7 +6386,7 @@ version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d3987094b1d07b653b7dfdc3f70ce9a1da9c51ac18c1b06b662e4f9a0e9f4b2" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", ] [[package]] @@ -6391,12 +6401,12 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.101.6" +version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c7d5dece342910d9ba34d259310cae3e0154b873b35408b787b59bce53d34fe" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring 0.16.20", - "untrusted 0.7.1", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -6467,12 +6477,12 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "sct" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" +checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.16.20", - "untrusted 0.7.1", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -6506,9 +6516,9 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.19" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad977052201c6de01a8ef2aa3378c4bd23217a056337d1d6da40468d267a4fb0" +checksum = "836fa6a3e1e547f9a2c4040802ec865b5d85f4014efe00555d7090a3dcaa1090" [[package]] name = "serde" @@ -6525,7 +6535,7 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" dependencies = [ - "ordered-float 2.10.0", + "ordered-float 2.10.1", "serde", ] @@ -6597,9 +6607,9 @@ dependencies = [ [[package]] name = "serde_spanned" -version = "0.6.3" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96426c9936fd7a0124915f9185ea1d20aa9445cc9821142f0a73bc9207a2e186" +checksum = "12022b835073e5b11e90a14f86838ceb1c8fb0325b72416845c487ac0fa95e80" dependencies = [ "serde", ] @@ -6632,7 +6642,7 @@ version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "64cd236ccc1b7a29e7e2739f27c0b2dd199804abc4290e32f59f3b68d6405c23" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "chrono", "hex", "indexmap 1.9.3", @@ -6857,9 +6867,9 @@ checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" -version = "0.4.9" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" +checksum = "9f7916fc008ca5542385b89a3d3ce689953c143e9304a9bf8beec1de48994c0d" dependencies = [ "libc", "winapi 0.3.9", @@ -6867,9 +6877,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4031e820eb552adee9295814c0ced9e5cf38ddf1e8b7d566d6de8e2538ea989e" +checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" dependencies = [ "libc", "windows-sys 0.48.0", @@ -6930,7 +6940,7 @@ version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8d6753e460c998bbd4cd8c6f0ed9a64346fcca0723d6e75e52fdc351c5d2169d" dependencies = [ - "ahash 0.8.3", + "ahash 0.8.6", "atoi", "byteorder", "bytes", @@ -6952,7 +6962,7 @@ dependencies = [ "once_cell", "paste", "percent-encoding", - "rustls 0.21.7", + "rustls 0.21.8", "rustls-pemfile", "serde", "serde_json", @@ -7014,8 +7024,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "864b869fdf56263f4c95c45483191ea0af340f9f3e3e7b4d57a61c7c87a970db" dependencies = [ "atoi", - "base64 0.21.4", - "bitflags 2.4.0", + "base64 0.21.5", + "bitflags 2.4.1", "byteorder", "bytes", "crc", @@ -7057,8 +7067,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "eb7ae0e6a97fb3ba33b23ac2671a5ce6e3cabe003f451abd5a56e7951d975624" dependencies = [ "atoi", - "base64 0.21.4", - "bitflags 2.4.0", + "base64 0.21.5", + "bitflags 2.4.1", "byteorder", "crc", "dotenvy", @@ -7262,7 +7272,7 @@ dependencies = [ "aho-corasick", "arc-swap", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "bitpacking", "byteorder", "census", @@ -7416,7 +7426,7 @@ dependencies = [ "cfg-if", "fastrand 2.0.1", "redox_syscall 0.3.5", - "rustix 0.38.17", + "rustix 0.38.20", "windows-sys 0.48.0", ] @@ -7448,18 +7458,18 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "thiserror" -version = "1.0.49" +version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1177e8c6d7ede7afde3585fd2513e611227efd6481bd78d2e82ba1ce16557ed4" +checksum = "f9a7210f5c9a7156bb50aa36aed4c95afb51df0df00713949448cf9e97d382d2" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.49" +version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10712f02019e9288794769fba95cd6847df9874d49d871d062172f9dd41bc4cc" +checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" dependencies = [ "proc-macro2", "quote", @@ -7601,7 +7611,7 @@ dependencies = [ "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.5.4", + "socket2 0.5.5", "tokio-macros", "tracing", "windows-sys 0.48.0", @@ -7666,7 +7676,7 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.7", + "rustls 0.21.8", "tokio", ] @@ -7696,16 +7706,16 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.9" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d68074620f57a0b21594d9735eb2e98ab38b17f80d3fcb189fca266771ca60d" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" dependencies = [ "bytes", "futures-core", "futures-io", "futures-sink", "futures-util", - "hashbrown 0.12.3", + "hashbrown 0.14.2", "pin-project-lite", "slab", "tokio", @@ -7735,9 +7745,9 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.6.3" +version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7cda73e2f1397b1262d6dfdcef8aafae14d1de7748d66822d3bfeeb6d03e5e4b" +checksum = "3550f4e9685620ac18a50ed434eb3aec30db8ba93b0287467bca5826ea25baf1" dependencies = [ "serde", ] @@ -7796,7 +7806,7 @@ dependencies = [ "async-stream", "async-trait", "axum", - "base64 0.21.4", + "base64 0.21.5", "bytes", "flate2", "futures-core", @@ -7860,7 +7870,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61c5bb1d698276a2443e5ecfabc1008bf15a36c12e6a7176e7bf089ea9131140" dependencies = [ "async-compression", - "bitflags 2.4.0", + "bitflags 2.4.1", "bytes", "futures-core", "futures-util", @@ -7888,11 +7898,10 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.37" +version = "0.1.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" dependencies = [ - "cfg-if", "log", "pin-project-lite", "tracing-attributes", @@ -7901,9 +7910,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", @@ -7912,9 +7921,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", "valuable", @@ -7932,12 +7941,12 @@ dependencies = [ [[package]] name = "tracing-log" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78ddad33d2d10b1ed7eb9d1f518a5674713876e97e5bb9b7345a7984fbb4f922" +checksum = "f751112709b4e791d8ce53e32c4ed2d353565a795ce84da2285393f41557bdf2" dependencies = [ - "lazy_static", "log", + "once_cell", "tracing-core", ] @@ -8241,7 +8250,7 @@ dependencies = [ "bytes", "chrono", "once_cell", - "ordered-float 3.9.1", + "ordered-float 3.9.2", "path", "regex", "serde", @@ -8285,7 +8294,7 @@ dependencies = [ "getrandom 0.2.10", "indoc", "lalrpop-util", - "ordered-float 3.9.1", + "ordered-float 3.9.2", "paste", "path", "regex", @@ -8308,7 +8317,7 @@ dependencies = [ "chrono-tz", "derivative", "nom", - "ordered-float 3.9.1", + "ordered-float 3.9.2", "path", "serde", "serde_json", @@ -8333,7 +8342,7 @@ source = "git+https://github.com/vectordotdev/vrl?rev=v0.3.0#113005bcee6cd7b5ea0 dependencies = [ "lalrpop", "lalrpop-util", - "ordered-float 3.9.1", + "ordered-float 3.9.2", "paste", "path", "thiserror", @@ -8347,7 +8356,7 @@ source = "git+https://github.com/vectordotdev/vrl?rev=v0.3.0#113005bcee6cd7b5ea0 dependencies = [ "aes", "base16", - "base64 0.21.4", + "base64 0.21.5", "bytes", "cbc", "cfb-mode", @@ -8368,7 +8377,7 @@ dependencies = [ "nom", "ofb", "once_cell", - "ordered-float 3.9.1", + "ordered-float 3.9.2", "path", "percent-encoding", "quoted_printable", @@ -8596,7 +8605,7 @@ version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" dependencies = [ - "ring 0.17.2", + "ring 0.17.5", "untrusted 0.9.0", ] @@ -8615,7 +8624,7 @@ version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b291546d5d9d1eab74f069c77749f2cb8504a12caa20f0f2de93ddbf6f411888" dependencies = [ - "rustls-webpki 0.101.6", + "rustls-webpki 0.101.7", ] [[package]] @@ -8633,7 +8642,7 @@ dependencies = [ "either", "home", "once_cell", - "rustix 0.38.17", + "rustix 0.38.20", ] [[package]] @@ -8699,6 +8708,15 @@ dependencies = [ "windows-targets 0.48.5", ] +[[package]] +name = "windows-core" +version = "0.51.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" +dependencies = [ + "windows-targets 0.48.5", +] + [[package]] name = "windows-sys" version = "0.45.0" @@ -8833,9 +8851,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "winnow" -version = "0.5.16" +version = "0.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "037711d82167854aff2018dfd193aa0fef5370f456732f0d5a0c59b0f1b4b907" +checksum = "a3b801d0e0a6726477cc207f60162da452f3a95adb368399bef20a946e06f65c" dependencies = [ "memchr", ] @@ -8858,7 +8876,7 @@ checksum = "c6f71803d3a1c80377a06221e0530be02035d5b3e854af56c6ece7ac20ac441d" dependencies = [ "assert-json-diff 2.0.2", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "deadpool", "futures", "futures-timer", @@ -8927,6 +8945,26 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" +[[package]] +name = "zerocopy" +version = "0.7.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69c48d63854f77746c68a5fbb4aa17f3997ece1cb301689a257af8cb80610d21" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c258c1040279e4f88763a113de72ce32dde2d50e2a94573f15dd534cea36a16d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + [[package]] name = "zeroize" version = "1.6.0" @@ -9011,11 +9049,10 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.8+zstd.1.5.5" +version = "2.0.9+zstd.1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5556e6ee25d32df2586c098bbfa278803692a20d0ab9565e049480d52707ec8c" +checksum = "9e16efa8a874a0481a574084d34cc26fdb3b99627480f785888deb6386506656" dependencies = [ "cc", - "libc", "pkg-config", ] diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 76aa223df5f..dd0ae829008 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -51,7 +51,7 @@ use quickwit_ingest::IngesterPool; use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::search::SearchResponse; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use quickwit_search::{single_node_search, SearchResponseRest}; use quickwit_serve::{ search_request_from_api_request, BodyFormat, SearchRequestQueryString, SortBy, diff --git a/quickwit/quickwit-cluster/src/change.rs b/quickwit/quickwit-cluster/src/change.rs index a3fe55667ec..268abe58599 100644 --- a/quickwit/quickwit-cluster/src/change.rs +++ b/quickwit/quickwit-cluster/src/change.rs @@ -23,7 +23,7 @@ use std::collections::BTreeMap; use chitchat::{ChitchatId, NodeState}; use quickwit_common::sorted_iter::{KeyDiff, SortedByKeyIterator}; use quickwit_common::tower::{make_channel, warmup_channel}; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use tonic::transport::Channel; use tracing::{info, warn}; diff --git a/quickwit/quickwit-cluster/src/cluster.rs b/quickwit/quickwit-cluster/src/cluster.rs index 54c1dab9a15..df5ec0c3f4e 100644 --- a/quickwit/quickwit-cluster/src/cluster.rs +++ b/quickwit/quickwit-cluster/src/cluster.rs @@ -32,7 +32,7 @@ use chitchat::{ use futures::Stream; use itertools::Itertools; use quickwit_proto::indexing::IndexingTask; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use serde::{Deserialize, Serialize}; use tokio::sync::{mpsc, watch, Mutex, RwLock}; use tokio::time::timeout; diff --git a/quickwit/quickwit-cluster/src/lib.rs b/quickwit/quickwit-cluster/src/lib.rs index 07f05cec88e..e1e4afc180c 100644 --- a/quickwit/quickwit-cluster/src/lib.rs +++ b/quickwit/quickwit-cluster/src/lib.rs @@ -30,7 +30,7 @@ use chitchat::transport::UdpTransport; use chitchat::FailureDetectorConfig; use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use time::OffsetDateTime; pub use crate::change::ClusterChange; diff --git a/quickwit/quickwit-cluster/src/member.rs b/quickwit/quickwit-cluster/src/member.rs index da4b8d10e93..4f9c8aec83e 100644 --- a/quickwit/quickwit-cluster/src/member.rs +++ b/quickwit/quickwit-cluster/src/member.rs @@ -24,7 +24,7 @@ use anyhow::{anyhow, Context}; use chitchat::{ChitchatId, NodeState}; use itertools::Itertools; use quickwit_proto::indexing::IndexingTask; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use tracing::warn; use crate::{GenerationId, QuickwitService}; diff --git a/quickwit/quickwit-config/src/index_config/mod.rs b/quickwit/quickwit-config/src/index_config/mod.rs index ed8e5cbd965..35fbcd5dd15 100644 --- a/quickwit/quickwit-config/src/index_config/mod.rs +++ b/quickwit/quickwit-config/src/index_config/mod.rs @@ -35,7 +35,7 @@ use quickwit_doc_mapper::{ DefaultDocMapper, DefaultDocMapperBuilder, DocMapper, FieldMappingEntry, Mode, ModeType, QuickwitJsonOptions, TokenizerEntry, }; -use quickwit_proto::IndexId; +use quickwit_proto::types::IndexId; use serde::{Deserialize, Serialize}; pub use serialize::load_index_config_from_user_config; diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 3db85b354c9..a23d2c26e0b 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -692,9 +692,9 @@ mod tests { subrequests: vec![GetOrCreateOpenShardsSubrequest { index_id: "test-index".to_string(), source_id: INGEST_SOURCE_ID.to_string(), - closed_shards: Vec::new(), }], - unavailable_ingesters: Vec::new(), + closed_shards: Vec::new(), + unavailable_leaders: Vec::new(), }; let get_open_shards_response = control_plane_mailbox .ask_for_res(get_open_shards_request) diff --git a/quickwit/quickwit-control-plane/src/control_plane_model.rs b/quickwit/quickwit-control-plane/src/control_plane_model.rs index f5a89fdadcc..98952293380 100644 --- a/quickwit/quickwit-control-plane/src/control_plane_model.rs +++ b/quickwit/quickwit-control-plane/src/control_plane_model.rs @@ -30,11 +30,12 @@ use quickwit_metastore::{ }; use quickwit_proto::control_plane::ControlPlaneResult; use quickwit_proto::ingest::{Shard, ShardState}; +use quickwit_proto::metastore; use quickwit_proto::metastore::{ EntityKind, ListIndexesMetadataRequest, ListShardsSubrequest, MetastoreError, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::{metastore, IndexId, IndexUid, NodeId, NodeIdRef, ShardId, SourceId}; +use quickwit_proto::types::{IndexId, IndexUid, NodeId, NodeIdRef, ShardId, SourceId}; use serde::Serialize; use tracing::{error, info}; @@ -87,7 +88,7 @@ impl ShardTableEntry { /// /// Upon starts, it loads its entire state from the metastore. #[derive(Default, Debug)] -pub struct ControlPlaneModel { +pub(crate) struct ControlPlaneModel { index_uid_table: FnvHashMap, index_table: FnvHashMap, shard_table: ShardTable, @@ -259,17 +260,31 @@ impl ControlPlaneModel { .delete_shards(index_uid, source_id, shard_ids); } + #[cfg(test)] + pub fn shards(&mut self) -> impl Iterator + '_ { + self.shard_table + .table_entries + .values() + .flat_map(|table_entry| table_entry.shards.values()) + } + + pub fn shards_mut(&mut self) -> impl Iterator + '_ { + self.shard_table + .table_entries + .values_mut() + .flat_map(|table_entry| table_entry.shards.values_mut()) + } + /// Sets the state of the shards identified by their index UID, source ID, and shard IDs to /// `Closed`. - #[allow(dead_code)] // Will remove this in a future PR. pub fn close_shards( &mut self, index_uid: &IndexUid, source_id: &SourceId, shard_ids: &[ShardId], - ) { + ) -> Vec { self.shard_table - .close_shards(index_uid, source_id, shard_ids); + .close_shards(index_uid, source_id, shard_ids) } pub fn index_uid(&self, index_id: &str) -> Option { @@ -426,13 +441,14 @@ impl ShardTable { /// Sets the state of the shards identified by their index UID, source ID, and shard IDs to /// `Closed`. - #[allow(dead_code)] // Will remove this in a future PR. pub fn close_shards( &mut self, index_uid: &IndexUid, source_id: &SourceId, shard_ids: &[ShardId], - ) { + ) -> Vec { + let mut closed_shard_ids = Vec::new(); + let source_uid = SourceUid { index_uid: index_uid.clone(), source_id: source_id.clone(), @@ -440,10 +456,14 @@ impl ShardTable { if let Some(table_entry) = self.table_entries.get_mut(&source_uid) { for shard_id in shard_ids { if let Some(shard) = table_entry.shards.get_mut(shard_id) { - shard.shard_state = ShardState::Closed as i32; + if !shard.is_closed() { + shard.shard_state = ShardState::Closed as i32; + closed_shard_ids.push(*shard_id); + } } } } + closed_shard_ids } /// Removes the shards identified by their index UID, source ID, and shard IDs. @@ -623,6 +643,53 @@ mod tests { assert_eq!(table_entry.next_shard_id, 3); } + #[test] + fn test_shard_table_close_shards() { + let index_uid_0: IndexUid = "test-index:0".into(); + let index_uid_1: IndexUid = "test-index:1".into(); + let source_id = "test-source".to_string(); + + let mut shard_table = ShardTable::default(); + + let shard_01 = Shard { + index_uid: index_uid_0.clone().into(), + source_id: source_id.clone(), + shard_id: 1, + leader_id: "test-leader-0".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }; + let shard_02 = Shard { + index_uid: index_uid_0.clone().into(), + source_id: source_id.clone(), + shard_id: 2, + leader_id: "test-leader-0".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }; + let shard_11 = Shard { + index_uid: index_uid_1.clone().into(), + source_id: source_id.clone(), + shard_id: 1, + leader_id: "test-leader-0".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }; + shard_table.update_shards(&index_uid_0, &source_id, &[shard_01, shard_02], 3); + shard_table.update_shards(&index_uid_0, &source_id, &[shard_11], 2); + + let closed_shard_ids = shard_table.close_shards(&index_uid_0, &source_id, &[1, 2, 3]); + assert_eq!(closed_shard_ids, &[1]); + + let source_uid_0 = SourceUid { + index_uid: index_uid_0, + source_id, + }; + let table_entry = shard_table.table_entries.get(&source_uid_0).unwrap(); + let shards = table_entry.shards(); + assert_eq!(shards[0].shard_state, ShardState::Closed as i32); + } + #[test] fn test_shard_table_delete_shards() { let index_uid_0: IndexUid = "test-index:0".into(); @@ -770,8 +837,4 @@ mod tests { assert_eq!(shards.len(), 0); assert_eq!(table_entry.next_shard_id, 1); } - #[tokio::test] - async fn test_ingest_controller_close_shards() { - // TODO: Write test when the RPC is actually called by ingesters. - } } diff --git a/quickwit/quickwit-control-plane/src/indexing_plan.rs b/quickwit/quickwit-control-plane/src/indexing_plan.rs index fa83c3b49bd..045900655b5 100644 --- a/quickwit/quickwit-control-plane/src/indexing_plan.rs +++ b/quickwit/quickwit-control-plane/src/indexing_plan.rs @@ -364,7 +364,7 @@ mod tests { }; use quickwit_metastore::IndexMetadata; use quickwit_proto::indexing::{IndexingServiceClient, IndexingTask}; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use rand::seq::SliceRandom; use serde_json::json; use tonic::transport::Endpoint; diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler.rs index 31f07574d0f..13bd1c8cabd 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler.rs @@ -26,7 +26,7 @@ use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; use quickwit_proto::indexing::{ApplyIndexingPlanRequest, IndexingService, IndexingTask}; use quickwit_proto::metastore::MetastoreServiceClient; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use serde::Serialize; use tracing::{debug, error, info, warn}; diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 5cb0f9a2801..b2a51957427 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -22,20 +22,22 @@ use std::time::Duration; use fnv::FnvHashSet; use itertools::Itertools; -use quickwit_common::Progress; +use quickwit_common::{PrettySample, Progress}; use quickwit_ingest::IngesterPool; use quickwit_proto::control_plane::{ - ControlPlaneError, ControlPlaneResult, GetOpenShardsSubresponse, GetOrCreateOpenShardsRequest, - GetOrCreateOpenShardsResponse, + ClosedShards, ControlPlaneError, ControlPlaneResult, GetOpenShardsSubresponse, + GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsResponse, }; use quickwit_proto::ingest::ingester::{IngesterService, PingRequest}; -use quickwit_proto::ingest::IngestV2Error; +use quickwit_proto::ingest::{IngestV2Error, ShardState}; +use quickwit_proto::metastore; use quickwit_proto::metastore::{ EntityKind, MetastoreError, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::{metastore, IndexUid, NodeId}; +use quickwit_proto::types::{IndexUid, NodeId}; use rand::seq::SliceRandom; use tokio::time::timeout; +use tracing::info; use crate::control_plane_model::ControlPlaneModel; @@ -171,6 +173,55 @@ impl IngestController { None } + fn handle_closed_shards( + &self, + closed_shards: Vec, + model: &mut ControlPlaneModel, + ) { + for closed_shard in closed_shards { + let index_uid: IndexUid = closed_shard.index_uid.into(); + let source_id = closed_shard.source_id; + let closed_shard_ids = + model.close_shards(&index_uid, &source_id, &closed_shard.shard_ids); + + if !closed_shard_ids.is_empty() { + info!( + index_id=%index_uid.index_id(), + source_id=%source_id, + shard_ids=?PrettySample::new(&closed_shard_ids, 5), + "closed {} shards reported by router", + closed_shard_ids.len() + ); + } + } + } + + fn handle_unavailable_leaders( + &self, + unavailable_leaders: &FnvHashSet, + model: &mut ControlPlaneModel, + ) { + let mut confirmed_unavailable_leaders = FnvHashSet::default(); + + for leader_id in unavailable_leaders { + if !self.ingester_pool.contains_key(leader_id) { + confirmed_unavailable_leaders.insert(leader_id.clone()); + } else { + // TODO: If a majority of ingesters consistenly reports a leader as unavailable, we + // should probably mark it as unavailable too. + } + } + if !confirmed_unavailable_leaders.is_empty() { + for shard in model.shards_mut() { + if shard.shard_state().is_open() + && confirmed_unavailable_leaders.contains(&shard.leader_id) + { + shard.shard_state = ShardState::Unavailable as i32; + } + } + } + } + /// Finds the open shards that satisfies the [`GetOrCreateOpenShardsRequest`] request sent by an /// ingest router. First, the control plane checks its internal shard table to find /// candidates. If it does not contain any, the control plane will ask @@ -184,12 +235,16 @@ impl IngestController { let mut get_open_shards_subresponses = Vec::with_capacity(get_open_shards_request.subrequests.len()); - let mut unavailable_ingesters: FnvHashSet = get_open_shards_request - .unavailable_ingesters + self.handle_closed_shards(get_open_shards_request.closed_shards, model); + + let mut unavailable_leaders: FnvHashSet = get_open_shards_request + .unavailable_leaders .into_iter() .map(|ingester_id| ingester_id.into()) .collect(); + self.handle_unavailable_leaders(&unavailable_leaders, model); + let mut open_shards_subrequests = Vec::new(); for get_open_shards_subrequest in get_open_shards_request.subrequests { @@ -205,7 +260,7 @@ impl IngestController { .find_open_shards( &index_uid, &get_open_shards_subrequest.source_id, - &unavailable_ingesters, + &unavailable_leaders, ) .ok_or_else(|| { MetastoreError::NotFound(EntityKind::Source { @@ -225,7 +280,7 @@ impl IngestController { // TODO: Find leaders in batches. // TODO: Round-robin leader-follower pairs or choose according to load. let (leader_id, follower_id) = self - .find_leader_and_follower(&mut unavailable_ingesters, progress) + .find_leader_and_follower(&mut unavailable_leaders, progress) .await .ok_or_else(|| { ControlPlaneError::Unavailable("no available ingester".to_string()) @@ -237,11 +292,9 @@ impl IngestController { follower_id: follower_id.map(|follower_id| follower_id.into()), next_shard_id, }; - open_shards_subrequests.push(open_shards_subrequest); } } - if !open_shards_subrequests.is_empty() { let open_shards_request = metastore::OpenShardsRequest { subrequests: open_shards_subrequests, @@ -289,6 +342,7 @@ mod tests { IngesterServiceClient, MockIngesterService, PingResponse, }; use quickwit_proto::ingest::{Shard, ShardState}; + use quickwit_proto::types::SourceId; use super::*; @@ -584,7 +638,8 @@ mod tests { let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), - unavailable_ingesters: Vec::new(), + closed_shards: Vec::new(), + unavailable_leaders: Vec::new(), }; let response = ingest_controller @@ -598,18 +653,18 @@ mod tests { GetOrCreateOpenShardsSubrequest { index_id: "test-index-0".to_string(), source_id: source_id.to_string(), - closed_shards: Vec::new(), }, GetOrCreateOpenShardsSubrequest { index_id: "test-index-1".to_string(), source_id: source_id.to_string(), - closed_shards: Vec::new(), }, ]; - let unavailable_ingesters = vec!["test-ingester-0".to_string()]; + let closed_shards = Vec::new(); + let unavailable_leaders = vec!["test-ingester-0".to_string()]; let request = GetOrCreateOpenShardsRequest { subrequests, - unavailable_ingesters, + closed_shards, + unavailable_leaders, }; let response = ingest_controller .get_or_create_open_shards(request, &mut model, &progress) @@ -638,4 +693,106 @@ mod tests { assert_eq!(model.observable_state().num_shards, 2); } + + #[tokio::test] + async fn test_ingest_controller_get_open_shards_handles_closed_shards() { + let metastore = MetastoreServiceClient::mock().into(); + let ingester_pool = IngesterPool::default(); + let replication_factor = 2; + + let mut ingest_controller = + IngestController::new(metastore, ingester_pool, replication_factor); + let mut model = ControlPlaneModel::default(); + + let index_uid: IndexUid = "test-index-0:0".into(); + let source_id: SourceId = "test-source".into(); + + let shards = vec![Shard { + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }]; + model.update_shards(&index_uid, &source_id, &shards, 3); + + let request = GetOrCreateOpenShardsRequest { + subrequests: Vec::new(), + closed_shards: vec![ClosedShards { + index_uid: index_uid.clone().into(), + source_id: source_id.clone(), + shard_ids: vec![1, 2], + }], + unavailable_leaders: Vec::new(), + }; + let progress = Progress::default(); + + ingest_controller + .get_or_create_open_shards(request, &mut model, &progress) + .await + .unwrap(); + + let shard_1 = model.shards().find(|shard| shard.shard_id == 1).unwrap(); + assert!(shard_1.shard_state().is_closed()); + } + + #[tokio::test] + async fn test_ingest_controller_get_open_shards_handles_unavailable_leaders() { + let metastore = MetastoreServiceClient::mock().into(); + + let ingester_pool = IngesterPool::default(); + let ingester_1 = IngesterServiceClient::mock().into(); + ingester_pool.insert("test-ingester-1".into(), ingester_1); + + let replication_factor = 2; + + let mut ingest_controller = + IngestController::new(metastore, ingester_pool.clone(), replication_factor); + let mut model = ControlPlaneModel::default(); + + let index_uid: IndexUid = "test-index-0:0".into(); + let source_id: SourceId = "test-source".into(); + + let shards = vec![ + Shard { + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + Shard { + shard_id: 2, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }, + Shard { + shard_id: 3, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + ]; + model.update_shards(&index_uid, &source_id, &shards, 4); + + let request = GetOrCreateOpenShardsRequest { + subrequests: Vec::new(), + closed_shards: Vec::new(), + unavailable_leaders: vec!["test-ingester-0".to_string()], + }; + let progress = Progress::default(); + + ingest_controller + .get_or_create_open_shards(request, &mut model, &progress) + .await + .unwrap(); + + let shard_1 = model.shards().find(|shard| shard.shard_id == 1).unwrap(); + assert!(shard_1.shard_state().is_unavailable()); + + let shard_2 = model.shards().find(|shard| shard.shard_id == 2).unwrap(); + assert!(shard_2.shard_state().is_closed()); + + let shard_3 = model.shards().find(|shard| shard.shard_id == 3).unwrap(); + assert!(shard_3.shard_state().is_open()); + } } diff --git a/quickwit/quickwit-control-plane/src/tests.rs b/quickwit/quickwit-control-plane/src/tests.rs index ec03c38a3f1..fcc99f660fc 100644 --- a/quickwit/quickwit-control-plane/src/tests.rs +++ b/quickwit/quickwit-control-plane/src/tests.rs @@ -35,7 +35,7 @@ use quickwit_proto::indexing::{ApplyIndexingPlanRequest, IndexingServiceClient}; use quickwit_proto::metastore::{ ListIndexesMetadataResponse, ListShardsResponse, MetastoreServiceClient, }; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use serde_json::json; use crate::control_plane::{ControlPlane, CONTROL_PLAN_LOOP_INTERVAL}; diff --git a/quickwit/quickwit-index-management/src/garbage_collection.rs b/quickwit/quickwit-index-management/src/garbage_collection.rs index fc661fe3fc6..b82b8ded214 100644 --- a/quickwit/quickwit-index-management/src/garbage_collection.rs +++ b/quickwit/quickwit-index-management/src/garbage_collection.rs @@ -32,7 +32,7 @@ use quickwit_proto::metastore::{ DeleteSplitsRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::{IndexUid, SplitId}; +use quickwit_proto::types::{IndexUid, SplitId}; use quickwit_storage::{BulkDeleteError, Storage}; use thiserror::Error; use time::OffsetDateTime; @@ -353,7 +353,7 @@ mod tests { use quickwit_proto::metastore::{ CreateIndexRequest, EntityKind, ListSplitsResponse, StageSplitsRequest, }; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use quickwit_storage::{ storage_for_test, BulkDeleteError, DeleteFailure, MockStorage, PutPayload, }; diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 62fc88a15de..20fad3bc20f 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -33,7 +33,8 @@ use quickwit_proto::metastore::{ ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, MetastoreServiceClient, ResetSourceCheckpointRequest, }; -use quickwit_proto::{IndexUid, ServiceError, ServiceErrorCode, SplitId}; +use quickwit_proto::types::{IndexUid, SplitId}; +use quickwit_proto::{ServiceError, ServiceErrorCode}; use quickwit_storage::{StorageResolver, StorageResolverError}; use thiserror::Error; use tracing::{error, info}; diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index c51c3419194..c36958dc128 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -54,7 +54,7 @@ use quickwit_metastore::{ }; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde_json::Value as JsonValue; use tantivy::{Directory, Inventory}; diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index 97cae5c890a..eca434b5c43 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -42,7 +42,7 @@ use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::{ LastDeleteOpstampRequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::PublishToken; +use quickwit_proto::types::PublishToken; use quickwit_query::get_quickwit_fastfield_normalizer_manager; use serde::Serialize; use tantivy::schema::Schema; @@ -649,7 +649,7 @@ mod tests { use quickwit_doc_mapper::{default_doc_mapper_for_test, DefaultDocMapper}; use quickwit_metastore::checkpoint::SourceCheckpointDelta; use quickwit_proto::metastore::{EmptyResponse, LastDeleteOpstampResponse}; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use tantivy::{doc, DateTime}; use super::*; diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index cd3cbd0560a..98bef9ac3f2 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -593,7 +593,7 @@ mod tests { EmptyResponse, IndexMetadataResponse, LastDeleteOpstampResponse, ListSplitsResponse, MetastoreError, }; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use quickwit_storage::RamStorage; use super::{IndexingPipeline, *}; diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 0860ff76db3..42c22d30f00 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -51,7 +51,7 @@ use quickwit_proto::indexing::{ use quickwit_proto::metastore::{ IndexMetadataRequest, ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::{IndexId, IndexUid}; +use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_storage::StorageResolver; use serde::{Deserialize, Serialize}; use tokio::sync::Semaphore; diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index 0294f4cd8a2..7b483ad3207 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -488,7 +488,7 @@ mod tests { use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::{ListSplitsResponse, MetastoreServiceClient}; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use quickwit_storage::RamStorage; use crate::actors::merge_pipeline::{MergePipeline, MergePipelineParams}; diff --git a/quickwit/quickwit-indexing/src/actors/merge_planner.rs b/quickwit/quickwit-indexing/src/actors/merge_planner.rs index 8387ff72ae6..99a2f18ec98 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_planner.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_planner.rs @@ -470,7 +470,7 @@ mod tests { use quickwit_config::IndexingSettings; use quickwit_metastore::{SplitMaturity, SplitMetadata}; use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use tantivy::TrackedObject; use time::OffsetDateTime; diff --git a/quickwit/quickwit-indexing/src/actors/packager.rs b/quickwit/quickwit-indexing/src/actors/packager.rs index 09c36e9935c..200aad9253b 100644 --- a/quickwit/quickwit-indexing/src/actors/packager.rs +++ b/quickwit/quickwit-indexing/src/actors/packager.rs @@ -338,7 +338,7 @@ mod tests { use quickwit_actors::{ObservationType, Universe}; use quickwit_metastore::checkpoint::IndexCheckpointDelta; use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use tantivy::directory::MmapDirectory; use tantivy::schema::{NumericOptions, Schema, FAST, STRING, TEXT}; use tantivy::{doc, DateTime, IndexBuilder, IndexSettings}; diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index 12c9c7b4205..4c5a769dfcd 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -197,7 +197,7 @@ mod tests { }; use quickwit_metastore::{PublishSplitsRequestExt, SplitMetadata}; use quickwit_proto::metastore::EmptyResponse; - use quickwit_proto::{IndexUid, Position}; + use quickwit_proto::types::{IndexUid, Position}; use tracing::Span; use super::*; diff --git a/quickwit/quickwit-indexing/src/merge_policy/mod.rs b/quickwit/quickwit-indexing/src/merge_policy/mod.rs index 1772ae6787b..9b1cfe34c29 100644 --- a/quickwit/quickwit-indexing/src/merge_policy/mod.rs +++ b/quickwit/quickwit-indexing/src/merge_policy/mod.rs @@ -170,7 +170,7 @@ pub mod tests { use proptest::prelude::*; use quickwit_actors::Universe; use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use rand::seq::SliceRandom; use tantivy::TrackedObject; use time::OffsetDateTime; diff --git a/quickwit/quickwit-indexing/src/models/mod.rs b/quickwit/quickwit-indexing/src/models/mod.rs index 922ba2823b2..230542ae60e 100644 --- a/quickwit/quickwit-indexing/src/models/mod.rs +++ b/quickwit/quickwit-indexing/src/models/mod.rs @@ -47,7 +47,7 @@ pub use packaged_split::{PackagedSplit, PackagedSplitBatch}; pub use processed_doc::{ProcessedDoc, ProcessedDocBatch}; pub use publish_lock::{NewPublishLock, PublishLock}; pub use publisher_message::SplitsUpdate; -use quickwit_proto::PublishToken; +use quickwit_proto::types::PublishToken; pub use raw_doc_batch::RawDocBatch; pub use split_attrs::{create_split_metadata, SplitAttrs}; diff --git a/quickwit/quickwit-indexing/src/source/file_source.rs b/quickwit/quickwit-indexing/src/source/file_source.rs index 9ca23c0f7b3..265e2f0e3d3 100644 --- a/quickwit/quickwit-indexing/src/source/file_source.rs +++ b/quickwit/quickwit-indexing/src/source/file_source.rs @@ -202,7 +202,7 @@ mod tests { use quickwit_config::{SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::checkpoint::{SourceCheckpoint, SourceCheckpointDelta}; use quickwit_metastore::metastore_for_test; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use super::*; use crate::source::SourceActor; diff --git a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs index 5402ab69049..4dc332d4e6b 100644 --- a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs +++ b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs @@ -297,7 +297,7 @@ mod gcp_pubsub_emulator_tests { use quickwit_actors::Universe; use quickwit_config::{SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::metastore_for_test; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use serde_json::json; use super::*; diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 2ed6c389be5..e218331580f 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -34,7 +34,7 @@ use quickwit_proto::ingest::ingester::{ use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsSubrequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::{IndexUid, NodeId, Position, PublishToken, ShardId, SourceId}; +use quickwit_proto::types::{IndexUid, NodeId, Position, PublishToken, ShardId, SourceId}; use serde_json::json; use tokio::time; use tracing::{debug, error, info, warn}; diff --git a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs index 49f7bbff10e..54ca2c4ceee 100644 --- a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs +++ b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs @@ -225,7 +225,7 @@ mod tests { use quickwit_ingest::{init_ingest_api, CommitType, DocBatchBuilder, IngestRequest}; use quickwit_metastore::checkpoint::{SourceCheckpoint, SourceCheckpointDelta}; use quickwit_metastore::metastore_for_test; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use super::*; use crate::source::SourceActor; diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index 0fde5b4fc98..908a488cba8 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -32,7 +32,7 @@ use quickwit_config::KafkaSourceParams; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService}; -use quickwit_proto::{IndexUid, Position}; +use quickwit_proto::types::{IndexUid, Position}; use rdkafka::config::{ClientConfig, RDKafkaLogLevel}; use rdkafka::consumer::{ BaseConsumer, CommitMode, Consumer, ConsumerContext, DefaultConsumerContext, Rebalance, @@ -795,7 +795,7 @@ mod kafka_broker_tests { CreateIndexRequest, MetastoreService, MetastoreServiceClient, PublishSplitsRequest, StageSplitsRequest, }; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use rdkafka::admin::{AdminClient, AdminOptions, NewTopic, TopicReplication}; use rdkafka::client::DefaultClientContext; use rdkafka::message::ToBytes; diff --git a/quickwit/quickwit-indexing/src/source/mod.rs b/quickwit/quickwit-indexing/src/source/mod.rs index 1ef016ecc95..09796f7f751 100644 --- a/quickwit/quickwit-indexing/src/source/mod.rs +++ b/quickwit/quickwit-indexing/src/source/mod.rs @@ -96,7 +96,7 @@ use quickwit_ingest::IngesterPool; use quickwit_metastore::checkpoint::{SourceCheckpoint, SourceCheckpointDelta}; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::MetastoreServiceClient; -use quickwit_proto::{IndexUid, ShardId}; +use quickwit_proto::types::{IndexUid, ShardId}; use quickwit_storage::StorageResolver; use serde_json::Value as JsonValue; pub use source_factory::{SourceFactory, SourceLoader, TypedSourceFactory}; diff --git a/quickwit/quickwit-indexing/src/source/source_factory.rs b/quickwit/quickwit-indexing/src/source/source_factory.rs index 1747e8a34cf..72aa013df9c 100644 --- a/quickwit/quickwit-indexing/src/source/source_factory.rs +++ b/quickwit/quickwit-indexing/src/source/source_factory.rs @@ -123,7 +123,7 @@ mod tests { use quickwit_config::{SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::metastore_for_test; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use super::*; use crate::source::quickwit_supported_sources; diff --git a/quickwit/quickwit-indexing/src/source/vec_source.rs b/quickwit/quickwit-indexing/src/source/vec_source.rs index ceb42d0547a..f23fe8a0bbe 100644 --- a/quickwit/quickwit-indexing/src/source/vec_source.rs +++ b/quickwit/quickwit-indexing/src/source/vec_source.rs @@ -138,7 +138,7 @@ mod tests { use quickwit_actors::{Actor, Command, Universe}; use quickwit_config::{SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::metastore_for_test; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use serde_json::json; use super::*; diff --git a/quickwit/quickwit-indexing/src/source/void_source.rs b/quickwit/quickwit-indexing/src/source/void_source.rs index fff9073a2cd..c4d204ea3d5 100644 --- a/quickwit/quickwit-indexing/src/source/void_source.rs +++ b/quickwit/quickwit-indexing/src/source/void_source.rs @@ -78,7 +78,7 @@ mod tests { use quickwit_config::{SourceInputFormat, SourceParams}; use quickwit_metastore::checkpoint::SourceCheckpoint; use quickwit_metastore::metastore_for_test; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use serde_json::json; use super::*; diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index fe6d36ddcbd..8327588cfd7 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -38,7 +38,7 @@ use quickwit_metastore::{ CreateIndexRequestExt, MetastoreResolver, Split, SplitMetadata, SplitState, }; use quickwit_proto::metastore::{CreateIndexRequest, MetastoreService, MetastoreServiceClient}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_storage::{Storage, StorageResolver}; use serde_json::Value as JsonValue; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingest.md b/quickwit/quickwit-ingest/src/ingest_v2/ingest.md new file mode 100644 index 00000000000..85bdf8d2d15 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingest.md @@ -0,0 +1,12 @@ +## Replication + +### Settings + +- ingest request timeout (30s), `Itimeout` +- persist request timeout (6s), `Ptimeout` +- replicate request timeout (3s), `Rtimeout` +- number of persist attempts (5), `k` + +Knowing that persist requests issue replicate requests, and ingest requests issue persist requests, we must have approximately: +- `Ptimeout` >= 2 * `Rtimeout` +- `Itimeout` >= `k` * `Ptimeout` diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index 2ecb940eae9..d6b27aaf352 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -35,7 +35,7 @@ use quickwit_common::ServiceStream; use quickwit_proto::ingest::ingester::{ AckReplicationMessage, FetchResponseV2, IngesterService, IngesterServiceClient, IngesterServiceStream, OpenFetchStreamRequest, OpenReplicationStreamRequest, - OpenReplicationStreamResponse, PersistFailure, PersistFailureKind, PersistRequest, + OpenReplicationStreamResponse, PersistFailure, PersistFailureReason, PersistRequest, PersistResponse, PersistSuccess, PingRequest, PingResponse, ReplicateRequest, ReplicateSubrequest, SynReplicationMessage, TruncateRequest, TruncateResponse, }; @@ -56,6 +56,14 @@ use crate::ingest_v2::models::SoloShard; use crate::metrics::INGEST_METRICS; use crate::{FollowerId, LeaderId}; +/// Duration after which persist requests time out with +/// [`quickwit_proto::ingest::IngestV2Error::Timeout`]. +pub(super) const PERSIST_REQUEST_TIMEOUT: Duration = if cfg!(any(test, feature = "testsuite")) { + Duration::from_millis(10) +} else { + Duration::from_secs(6) +}; + #[derive(Clone)] pub struct Ingester { self_node_id: NodeId, @@ -262,7 +270,7 @@ impl IngesterService for Ingester { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - failure_kind: PersistFailureKind::ShardClosed as i32, + failure_reason: PersistFailureReason::ShardClosed as i32, }; persist_failures.push(persist_failure); continue; @@ -363,7 +371,7 @@ impl IngesterService for Ingester { // TODO: Handle replication error: // 1. Close and evict all the shards hosted by the follower. // 2. Close and evict the replication client. - // 3. Return `PersistFailureKind::ShardClose` to router. + // 3. Return `PersistFailureReason::ShardClosed` to router. continue; } }; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index f9931e8fe46..53ab623fbfb 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -29,7 +29,7 @@ mod test_utils; use quickwit_common::tower::Pool; use quickwit_proto::ingest::ingester::IngesterServiceClient; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; pub use self::fetch::MultiFetchStream; pub use self::ingester::Ingester; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 4b30096e306..ede0979f181 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -42,6 +42,7 @@ use crate::metrics::INGEST_METRICS; pub(super) const SYN_REPLICATION_STREAM_CAPACITY: usize = 5; +/// Duration after which replication requests time out with [`ReplicationError::Timeout`]. const REPLICATION_REQUEST_TIMEOUT: Duration = if cfg!(any(test, feature = "testsuite")) { Duration::from_millis(10) } else { @@ -205,6 +206,8 @@ pub(super) struct ReplicationStreamTaskHandle { } impl ReplicationStreamTaskHandle { + /// Enqueues a replication request into the replication stream and waits for the response. Times + /// out after [`REPLICATION_REQUEST_TIMEOUT`] seconds. pub fn replicate( &self, replicate_request: ReplicateRequest, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 8528ebc71fe..f05da8feb9d 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -17,28 +17,39 @@ // 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::collections::{HashMap, HashSet}; use std::fmt; use std::sync::Arc; +use std::time::Duration; use async_trait::async_trait; use futures::stream::FuturesUnordered; -use futures::StreamExt; +use futures::{Future, StreamExt}; use quickwit_proto::control_plane::{ - ControlPlaneService, ControlPlaneServiceClient, GetOrCreateOpenShardsRequest, + ClosedShards, ControlPlaneService, ControlPlaneServiceClient, GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsSubrequest, }; -use quickwit_proto::ingest::ingester::{IngesterService, PersistRequest, PersistSubrequest}; +use quickwit_proto::ingest::ingester::{ + IngesterService, PersistFailureReason, PersistRequest, PersistResponse, PersistSubrequest, +}; use quickwit_proto::ingest::router::{ IngestRequestV2, IngestResponseV2, IngestRouterService, IngestSubrequest, }; -use quickwit_proto::ingest::IngestV2Result; -use quickwit_proto::types::{IndexUid, NodeId}; +use quickwit_proto::ingest::{IngestV2Error, IngestV2Result}; +use quickwit_proto::types::{IndexUid, NodeId, ShardId, SourceId}; use tokio::sync::RwLock; +use super::ingester::PERSIST_REQUEST_TIMEOUT; use super::shard_table::ShardTable; use super::IngesterPool; +/// Duration after which ingest requests time out with [`IngestV2Error::Timeout`]. +pub(super) const INGEST_REQUEST_TIMEOUT: Duration = if cfg!(any(test, feature = "testsuite")) { + Duration::from_millis(10) +} else { + Duration::from_secs(35) +}; + type LeaderId = String; #[derive(Clone)] @@ -90,19 +101,26 @@ impl IngestRouter { async fn make_get_or_create_open_shard_request( &self, subrequests: &[IngestSubrequest], + ingester_pool: &IngesterPool, ) -> GetOrCreateOpenShardsRequest { let state_guard = self.state.read().await; let mut get_open_shards_subrequests = Vec::new(); + // `closed_shards` and `unavailable_leaders` are populated by calls to `has_open_shards` + // as we're looking for open shards to route the subrequests to. + let mut closed_shards: Vec = Vec::new(); + let mut unavailable_leaders: HashSet = HashSet::new(); for subrequest in subrequests { - if !state_guard - .shard_table - .contains_entry(&subrequest.index_id, &subrequest.source_id) - { + if !state_guard.shard_table.has_open_shards( + &subrequest.index_id, + &subrequest.source_id, + &mut closed_shards, + ingester_pool, + &mut unavailable_leaders, + ) { let subrequest = GetOrCreateOpenShardsSubrequest { index_id: subrequest.index_id.clone(), source_id: subrequest.source_id.clone(), - closed_shards: Vec::new(), // TODO }; get_open_shards_subrequests.push(subrequest); } @@ -110,7 +128,8 @@ impl IngestRouter { GetOrCreateOpenShardsRequest { subrequests: get_open_shards_subrequests, - unavailable_ingesters: Vec::new(), + closed_shards, + unavailable_leaders: unavailable_leaders.into_iter().map(Into::into).collect(), } } @@ -131,26 +150,59 @@ impl IngestRouter { let mut state_guard = self.state.write().await; for subresponse in response.subresponses { - let index_uid: IndexUid = subresponse.index_uid.into(); - let index_id = index_uid.index_id().to_string(); state_guard.shard_table.insert_shards( - index_id, + subresponse.index_uid, subresponse.source_id, subresponse.open_shards, ); } Ok(()) } -} -#[async_trait] -impl IngestRouterService for IngestRouter { - async fn ingest( + async fn process_persist_results( + &mut self, + mut persist_futures: FuturesUnordered< + impl Future>, + >, + ) { + let mut closed_shards: HashMap<(IndexUid, SourceId), Vec> = HashMap::new(); + + while let Some(persist_result) = persist_futures.next().await { + match persist_result { + Ok(persist_response) => { + for persist_failure in persist_response.failures { + if persist_failure.failure_reason() == PersistFailureReason::ShardClosed { + let index_uid: IndexUid = persist_failure.index_uid.into(); + let source_id: SourceId = persist_failure.source_id; + closed_shards + .entry((index_uid, source_id)) + .or_default() + .push(persist_failure.shard_id); + } + } + } + Err(_persist_error) => { + // TODO + } + }; + } + if !closed_shards.is_empty() { + let mut state_guard = self.state.write().await; + + for ((index_uid, source_id), shard_ids) in closed_shards { + state_guard + .shard_table + .close_shards(&index_uid, source_id, &shard_ids); + } + } + } + + async fn ingest_attempt_one( &mut self, ingest_request: IngestRequestV2, ) -> IngestV2Result { let get_or_create_open_shards_request = self - .make_get_or_create_open_shard_request(&ingest_request.subrequests) + .make_get_or_create_open_shard_request(&ingest_request.subrequests, &self.ingester_pool) .await; self.populate_shard_table(get_or_create_open_shards_request) .await?; @@ -168,7 +220,8 @@ impl IngestRouterService for IngestRouter { .shard_table .find_entry(&ingest_subrequest.index_id, &ingest_subrequest.source_id) .expect("TODO") - .next_shard_round_robin(); + .next_open_shard_round_robin(&self.ingester_pool) + .expect("TODO"); let persist_subrequest = PersistSubrequest { index_uid: shard.index_uid.clone(), @@ -182,7 +235,7 @@ impl IngestRouterService for IngestRouter { .or_default() .push(persist_subrequest); } - let mut persist_futures = FuturesUnordered::new(); + let persist_futures = FuturesUnordered::new(); for (leader_id, subrequests) in per_leader_persist_subrequests { let leader_id: NodeId = leader_id.clone().into(); @@ -193,30 +246,66 @@ impl IngestRouterService for IngestRouter { subrequests, commit_type: ingest_request.commit_type, }; - let persist_future = async move { ingester.persist(persist_request).await }; + let persist_future = async move { + tokio::time::timeout(PERSIST_REQUEST_TIMEOUT, ingester.persist(persist_request)) + .await + .map_err(|_| IngestV2Error::Timeout)? + }; persist_futures.push(persist_future); } drop(state_guard); - while let Some(persist_result) = persist_futures.next().await { - // TODO: Handle errors. - persist_result?; - } + self.process_persist_results(persist_futures).await; + Ok(IngestResponseV2 { - successes: Vec::new(), // TODO - failures: Vec::new(), + subresponses: Vec::new(), }) } + + async fn ingest_attempt_many( + &mut self, + ingest_request: IngestRequestV2, + ) -> IngestV2Result { + // TODO: Introduce IngestRequestWorbench and implement retry logic and perform multiple + // attempts. + self.ingest_attempt_one(ingest_request).await + } + + async fn ingest_timeout( + &mut self, + ingest_request: IngestRequestV2, + timeout_duration: Duration, + ) -> IngestV2Result { + tokio::time::timeout(timeout_duration, self.ingest_attempt_many(ingest_request)) + .await + .map_err(|_| IngestV2Error::Timeout)? + } +} + +#[async_trait] +impl IngestRouterService for IngestRouter { + async fn ingest( + &mut self, + ingest_request: IngestRequestV2, + ) -> IngestV2Result { + self.ingest_timeout(ingest_request, INGEST_REQUEST_TIMEOUT) + .await + } } #[cfg(test)] mod tests { + use std::sync::atomic::AtomicUsize; + use quickwit_proto::control_plane::{GetOpenShardsSubresponse, GetOrCreateOpenShardsResponse}; - use quickwit_proto::ingest::ingester::{IngesterServiceClient, PersistResponse}; + use quickwit_proto::ingest::ingester::{ + IngesterServiceClient, PersistFailure, PersistResponse, + }; use quickwit_proto::ingest::router::IngestSubrequest; - use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2, Shard}; + use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2, Shard, ShardState}; use super::*; + use crate::ingest_v2::shard_table::ShardTableEntry; #[tokio::test] async fn test_router_make_get_or_create_open_shard_request() { @@ -230,10 +319,43 @@ mod tests { ingester_pool.clone(), replication_factor, ); - let get_or_create_open_shard_request = - router.make_get_or_create_open_shard_request(&[]).await; + let get_or_create_open_shard_request = router + .make_get_or_create_open_shard_request(&[], &ingester_pool) + .await; assert!(get_or_create_open_shard_request.subrequests.is_empty()); + let mut state_guard = router.state.write().await; + + state_guard.shard_table.table.insert( + ("test-index-0".into(), "test-source".into()), + ShardTableEntry { + index_uid: "test-index-0:0".into(), + source_id: "test-source".to_string(), + local_shards: vec![ + Shard { + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 2, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + ], + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), + remote_round_robin_idx: AtomicUsize::default(), + }, + ); + drop(state_guard); + let ingest_subrequests = [ IngestSubrequest { index_id: "test-index-0".to_string(), @@ -247,7 +369,7 @@ mod tests { }, ]; let get_or_create_open_shard_request = router - .make_get_or_create_open_shard_request(&ingest_subrequests) + .make_get_or_create_open_shard_request(&ingest_subrequests, &ingester_pool) .await; assert_eq!(get_or_create_open_shard_request.subrequests.len(), 2); @@ -260,29 +382,42 @@ mod tests { assert_eq!(subrequest.index_id, "test-index-1"); assert_eq!(subrequest.source_id, "test-source"); - let mut state_guard = router.state.write().await; - - state_guard.shard_table.insert_shards( - "test-index-0", - "test-source", - vec![Shard { - index_uid: "test-index-0:0".to_string(), + assert_eq!(get_or_create_open_shard_request.closed_shards.len(), 1); + assert_eq!( + get_or_create_open_shard_request.closed_shards[0], + ClosedShards { + index_uid: "test-index-0:0".into(), source_id: "test-source".to_string(), - shard_id: 0, - ..Default::default() - }], + shard_ids: vec![1], + } + ); + assert_eq!( + get_or_create_open_shard_request.unavailable_leaders.len(), + 1 + ); + assert_eq!( + get_or_create_open_shard_request.unavailable_leaders[0], + "test-ingester-0" + ); + + ingester_pool.insert( + "test-ingester-0".into(), + IngesterServiceClient::mock().into(), ); - drop(state_guard); let get_or_create_open_shard_request = router - .make_get_or_create_open_shard_request(&ingest_subrequests) + .make_get_or_create_open_shard_request(&ingest_subrequests, &ingester_pool) .await; - assert_eq!(get_or_create_open_shard_request.subrequests.len(), 1); - let subrequest = &get_or_create_open_shard_request.subrequests[0]; assert_eq!(subrequest.index_id, "test-index-1"); assert_eq!(subrequest.source_id, "test-source"); + + assert_eq!(get_or_create_open_shard_request.subrequests.len(), 1); + assert_eq!( + get_or_create_open_shard_request.unavailable_leaders.len(), + 0 + ); } #[tokio::test] @@ -347,7 +482,8 @@ mod tests { ); let get_or_create_open_shards_request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), - unavailable_ingesters: Vec::new(), + closed_shards: Vec::new(), + unavailable_leaders: Vec::new(), }; router .populate_shard_table(get_or_create_open_shards_request) @@ -360,20 +496,18 @@ mod tests { GetOrCreateOpenShardsSubrequest { index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), - ..Default::default() }, GetOrCreateOpenShardsSubrequest { index_id: "test-index-1".to_string(), source_id: "test-source".to_string(), - ..Default::default() }, GetOrCreateOpenShardsSubrequest { index_id: "test-index-2".to_string(), source_id: "test-source".to_string(), - ..Default::default() }, ], - unavailable_ingesters: Vec::new(), + closed_shards: Vec::new(), + unavailable_leaders: Vec::new(), }; router .populate_shard_table(get_or_create_open_shards_request) @@ -399,7 +533,7 @@ mod tests { } #[tokio::test] - async fn test_router_ingest() { + async fn test_router_process_persist_results_closes_shards() { let self_node_id = "test-router".into(); let control_plane = ControlPlaneServiceClient::mock().into(); let ingester_pool = IngesterPool::default(); @@ -410,7 +544,60 @@ mod tests { ingester_pool.clone(), replication_factor, ); + let mut state_guard = router.state.write().await; + state_guard.shard_table.insert_shards( + "test-index-0:0", + "test-source", + vec![Shard { + index_uid: "test-index-0:0".to_string(), + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + ..Default::default() + }], + ); + drop(state_guard); + + let persist_futures = FuturesUnordered::new(); + + persist_futures.push(async { + Ok::<_, IngestV2Error>(PersistResponse { + leader_id: "test-ingester-0".to_string(), + successes: Vec::new(), + failures: vec![PersistFailure { + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + failure_reason: PersistFailureReason::ShardClosed as i32, + }], + }) + }); + router.process_persist_results(persist_futures).await; + let state_guard = router.state.read().await; + let shard_table_entry = state_guard + .shard_table + .find_entry("test-index-0", "test-source") + .unwrap(); + assert_eq!(shard_table_entry.len(), 1); + assert_eq!(shard_table_entry.shards()[0].shard_id, 1); + assert_eq!( + shard_table_entry.shards()[0].shard_state, + ShardState::Closed as i32 + ); + } + + #[tokio::test] + async fn test_router_ingest() { + let self_node_id = "test-router".into(); + let control_plane = ControlPlaneServiceClient::mock().into(); + let ingester_pool = IngesterPool::default(); + let replication_factor = 1; + let mut router = IngestRouter::new( + self_node_id, + control_plane, + ingester_pool.clone(), + replication_factor, + ); let mut state_guard = router.state.write().await; state_guard.shard_table.insert_shards( "test-index-0", diff --git a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs index 88e24f9f57e..3f5ba077c59 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs @@ -17,58 +17,135 @@ // 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::collections::{HashMap, HashSet}; use std::sync::atomic::{AtomicUsize, Ordering}; -use quickwit_proto::ingest::Shard; -use quickwit_proto::types::{IndexId, NodeId, SourceId}; +use quickwit_proto::control_plane::ClosedShards; +use quickwit_proto::ingest::{Shard, ShardState}; +use quickwit_proto::types::{IndexId, IndexUid, NodeId, ShardId, SourceId}; +use tracing::warn; -/// A set of open shards for a given index and source. +use crate::IngesterPool; + +/// The set of shards the router is aware of for the given index and source. #[derive(Debug, Default)] pub(super) struct ShardTableEntry { - local_shards: Vec, - local_shards_round_robin_idx: AtomicUsize, - remote_shards: Vec, - remote_shards_round_robin_idx: AtomicUsize, + /// The index UID of the shards. + pub index_uid: IndexUid, + pub source_id: SourceId, + pub local_shards: Vec, + pub local_round_robin_idx: AtomicUsize, + pub remote_shards: Vec, + pub remote_round_robin_idx: AtomicUsize, } impl ShardTableEntry { - /// Creates a new entry and ensures that the shards are open and unique. - /// - /// # Panics - /// - /// Panics if `shards` is empty after filtering out closed shards and deduplicating by shard ID. - pub fn new(mut shards: Vec, node_id: &NodeId) -> Self { - shards.retain(|shard| shard.is_open()); + /// Creates a new entry and ensures that the shards are open, unique, and sorted by shard ID. + pub fn new( + self_node_id: &NodeId, + index_uid: IndexUid, + source_id: SourceId, + mut shards: Vec, + ) -> Self { + let num_shards = shards.len(); + shards.sort_unstable_by_key(|shard| shard.shard_id); shards.dedup_by_key(|shard| shard.shard_id); - assert!(!shards.is_empty(), "`shards` should not be empty"); - - let (local_shards, remote_shards) = shards + let (local_shards, remote_shards): (Vec<_>, Vec<_>) = shards .into_iter() - .partition(|shard| node_id == &shard.leader_id.as_str()); + .filter(|shard| shard.is_open()) + .partition(|shard| *self_node_id == shard.leader_id); + + if num_shards > local_shards.len() + remote_shards.len() { + warn!("input shards should not contain closed shards or duplicates"); + } Self { + index_uid, + source_id, local_shards, - local_shards_round_robin_idx: AtomicUsize::default(), + local_round_robin_idx: AtomicUsize::default(), remote_shards, - remote_shards_round_robin_idx: AtomicUsize::default(), + remote_round_robin_idx: AtomicUsize::default(), } } - /// Returns the next shard in round-robin order. - pub fn next_shard_round_robin(&self) -> &Shard { - if !self.local_shards.is_empty() { - let shard_idx = self - .local_shards_round_robin_idx - .fetch_add(1, Ordering::Relaxed); - &self.local_shards[shard_idx % self.local_shards.len()] - } else { - let shard_idx = self - .remote_shards_round_robin_idx - .fetch_add(1, Ordering::Relaxed); - &self.remote_shards[shard_idx % self.remote_shards.len()] + /// Returns `true` if at least one shard in the table entry is open and has a leader available. + /// As it goes through the list of shards in the entry, it populates `closed_shard_ids` and + /// `unavailable_leaders` with the shard IDs of the closed shards and the node ID of the + /// unavailable ingesters encountered along the way. + pub fn has_open_shards( + &self, + closed_shard_ids: &mut Vec, + ingester_pool: &IngesterPool, + unavailable_leaders: &mut HashSet, + ) -> bool { + for shards in [&self.local_shards, &self.remote_shards] { + for shard in shards { + if shard.is_closed() { + closed_shard_ids.push(shard.shard_id); + } else if shard.is_open() { + if ingester_pool.contains_key(&shard.leader_id) { + return true; + } else { + let leader_id: NodeId = shard.leader_id.clone().into(); + unavailable_leaders.insert(leader_id); + } + } + } + } + false + } + + /// Returns the next open and available shard in the table entry in a round-robin fashion. + pub fn next_open_shard_round_robin(&self, ingester_pool: &IngesterPool) -> Option<&Shard> { + for (shards, round_robin_idx) in [ + (&self.local_shards, &self.local_round_robin_idx), + (&self.remote_shards, &self.remote_round_robin_idx), + ] { + if shards.is_empty() { + continue; + } + let mut num_attempts = 0; + let max_num_attempts = shards.len(); + + while num_attempts < max_num_attempts { + let shard_idx = round_robin_idx.fetch_add(1, Ordering::Relaxed); + let shard = &shards[shard_idx % shards.len()]; + + if shard.is_open() && ingester_pool.contains_key(&shard.leader_id) { + return Some(shard); + } + num_attempts += 1; + } + } + None + } + + /// Closes the shards identified by their shard IDs. + pub fn close_shards(&mut self, index_uid: &IndexUid, shard_ids: &[ShardId]) { + // If the shard table was just recently updated with shards for a new index UID, then we can + // safely discard this request. + if self.index_uid != *index_uid { + return; + } + for shards in [&mut self.local_shards, &mut self.remote_shards] { + if shards.is_empty() { + continue; + } + let num_shards = shards.len(); + let shard_ids_range = shards[0].shard_id..=shards[num_shards - 1].shard_id; + + for shard_id in shard_ids { + if !shard_ids_range.contains(shard_id) { + continue; + } + if let Ok(shard_idx) = shards.binary_search_by_key(shard_id, |shard| shard.shard_id) + { + shards[shard_idx].shard_state = ShardState::Closed as i32; + } + } } } @@ -86,41 +163,77 @@ impl ShardTableEntry { } } -/// A table of shard entries indexed by index UID and source ID. +/// Stores the list of shards the router is aware of for each index and source. The resolution from +/// index and source to shards is performed using index ID (not index UID) and source ID. #[derive(Debug)] pub(super) struct ShardTable { - pub(super) table: HashMap<(IndexId, SourceId), ShardTableEntry>, - pub(super) self_node_id: NodeId, + pub self_node_id: NodeId, + pub table: HashMap<(IndexId, SourceId), ShardTableEntry>, } impl ShardTable { - pub fn contains_entry( + pub fn find_entry( &self, index_id: impl Into, source_id: impl Into, - ) -> bool { + ) -> Option<&ShardTableEntry> { let key = (index_id.into(), source_id.into()); - self.table.contains_key(&key) + self.table.get(&key) } - pub fn find_entry( + pub fn has_open_shards( &self, index_id: impl Into, source_id: impl Into, - ) -> Option<&ShardTableEntry> { - let key = (index_id.into(), source_id.into()); - self.table.get(&key) + closed_shards: &mut Vec, + ingester_pool: &IngesterPool, + unavailable_leaders: &mut HashSet, + ) -> bool { + if let Some(entry) = self.find_entry(index_id, source_id) { + let mut closed_shard_ids: Vec = Vec::new(); + + let result = + entry.has_open_shards(&mut closed_shard_ids, ingester_pool, unavailable_leaders); + + if !closed_shard_ids.is_empty() { + closed_shards.push(ClosedShards { + index_uid: entry.index_uid.clone().into(), + source_id: entry.source_id.clone(), + shard_ids: closed_shard_ids, + }); + } + result + } else { + false + } + } + + pub fn close_shards( + &mut self, + index_uid: &IndexUid, + source_id: impl Into, + shard_ids: &[ShardId], + ) { + let key = (index_uid.index_id().into(), source_id.into()); + if let Some(entry) = self.table.get_mut(&key) { + entry.close_shards(index_uid, shard_ids); + } } pub fn insert_shards( &mut self, - index_id: impl Into, + index_uid: impl Into, source_id: impl Into, shards: Vec, ) { - let key = (index_id.into(), source_id.into()); - self.table - .insert(key, ShardTableEntry::new(shards, &self.self_node_id)); + let index_uid: IndexUid = index_uid.into(); + let index_id: IndexId = index_uid.index_id().into(); + let source_id: SourceId = source_id.into(); + let key = (index_id, source_id.clone()); + self.table.insert( + key, + ShardTableEntry::new(&self.self_node_id, index_uid, source_id, shards), + ); } #[cfg(test)] @@ -136,106 +249,380 @@ impl ShardTable { #[cfg(test)] mod tests { + use quickwit_proto::ingest::ingester::IngesterServiceClient; use quickwit_proto::ingest::ShardState; use super::*; #[test] - fn test_shard_table_with_nonempty_local_shards() { - let mut table = ShardTable { - self_node_id: "node-0".into(), - table: HashMap::default(), + fn test_shard_table_entry_new() { + let self_node_id: NodeId = "test-node-0".into(); + let index_uid: IndexUid = IndexUid::new_2("test-index", 0); + let source_id: SourceId = "test-source".into(); + let table_entry = ShardTableEntry::new( + &self_node_id, + index_uid.clone(), + source_id.clone(), + Vec::new(), + ); + assert_eq!(table_entry.len(), 0); + + let shards = vec![ + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 3, + shard_state: ShardState::Open as i32, + leader_id: "test-node-0".to_string(), + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 1, + shard_state: ShardState::Open as i32, + leader_id: "test-node-0".to_string(), + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 2, + shard_state: ShardState::Open as i32, + leader_id: "test-node-1".to_string(), + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 1, + shard_state: ShardState::Open as i32, + leader_id: "test-node-0".to_string(), + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 4, + shard_state: ShardState::Closed as i32, + leader_id: "test-node-0".to_string(), + ..Default::default() + }, + ]; + let table_entry = ShardTableEntry::new(&self_node_id, index_uid, source_id, shards); + assert_eq!(table_entry.local_shards.len(), 2); + assert_eq!(table_entry.local_shards[0].shard_id, 1); + assert_eq!(table_entry.local_shards[1].shard_id, 3); + + assert_eq!(table_entry.remote_shards.len(), 1); + assert_eq!(table_entry.remote_shards[0].shard_id, 2); + } + + #[test] + fn test_shard_table_entry_has_open_shards() { + let index_uid: IndexUid = IndexUid::new_2("test-index", 0); + let source_id: SourceId = "test-source".into(); + let table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: Vec::new(), + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), + remote_round_robin_idx: AtomicUsize::default(), }; - assert!(!table.contains_entry("test-index", "test-source")); + let mut closed_shard_ids = Vec::new(); + let ingester_pool = IngesterPool::default(); + let mut unavailable_leaders = HashSet::new(); + + assert!(!table_entry.has_open_shards( + &mut closed_shard_ids, + &ingester_pool, + &mut unavailable_leaders + )); + assert!(closed_shard_ids.is_empty()); + assert!(unavailable_leaders.is_empty()); + + ingester_pool.insert( + "test-ingester-0".into(), + IngesterServiceClient::mock().into(), + ); + ingester_pool.insert( + "test-ingester-1".into(), + IngesterServiceClient::mock().into(), + ); - table.insert_shards( - "test-index", - "test-source", - vec![ + let table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: vec![ + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 2, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + ], + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), + remote_round_robin_idx: AtomicUsize::default(), + }; + assert!(table_entry.has_open_shards( + &mut closed_shard_ids, + &ingester_pool, + &mut unavailable_leaders + )); + assert_eq!(closed_shard_ids.len(), 1); + assert_eq!(closed_shard_ids[0], 1); + assert!(unavailable_leaders.is_empty()); + + closed_shard_ids.clear(); + + let table_entry = ShardTableEntry { + index_uid, + source_id, + local_shards: Vec::new(), + local_round_robin_idx: AtomicUsize::default(), + remote_shards: vec![ + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 1, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }, Shard { index_uid: "test-index:0".to_string(), - shard_id: 0, - leader_id: "node-0".to_string(), + shard_id: 2, + leader_id: "test-ingester-2".to_string(), + shard_state: ShardState::Open as i32, ..Default::default() }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 3, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + ], + remote_round_robin_idx: AtomicUsize::default(), + }; + assert!(table_entry.has_open_shards( + &mut closed_shard_ids, + &ingester_pool, + &mut unavailable_leaders + )); + assert_eq!(closed_shard_ids.len(), 1); + assert_eq!(closed_shard_ids[0], 1); + assert_eq!(unavailable_leaders.len(), 1); + assert!(unavailable_leaders.contains("test-ingester-2")); + } + + #[test] + fn test_shard_table_entry_next_open_shard_round_robin() { + let index_uid: IndexUid = IndexUid::new_2("test-index", 0); + let source_id: SourceId = "test-source".into(); + let table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: Vec::new(), + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), + remote_round_robin_idx: AtomicUsize::default(), + }; + let ingester_pool = IngesterPool::default(); + + let shard_opt = table_entry.next_open_shard_round_robin(&ingester_pool); + assert!(shard_opt.is_none()); + + ingester_pool.insert( + "test-ingester-0".into(), + IngesterServiceClient::mock().into(), + ); + ingester_pool.insert( + "test-ingester-1".into(), + IngesterServiceClient::mock().into(), + ); + + let table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: vec![ Shard { index_uid: "test-index:0".to_string(), shard_id: 1, - leader_id: "node-1".to_string(), + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 2, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, ..Default::default() }, Shard { index_uid: "test-index:0".to_string(), - shard_id: 0, - leader_id: "node-0".to_string(), + shard_id: 3, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, ..Default::default() }, + ], + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), + remote_round_robin_idx: AtomicUsize::default(), + }; + let shard = table_entry + .next_open_shard_round_robin(&ingester_pool) + .unwrap(); + assert_eq!(shard.shard_id, 2); + + let shard = table_entry + .next_open_shard_round_robin(&ingester_pool) + .unwrap(); + assert_eq!(shard.shard_id, 3); + + let shard = table_entry + .next_open_shard_round_robin(&ingester_pool) + .unwrap(); + assert_eq!(shard.shard_id, 2); + + let table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: vec![Shard { + index_uid: "test-index:0".to_string(), + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Closed as i32, + ..Default::default() + }], + local_round_robin_idx: AtomicUsize::default(), + remote_shards: vec![ Shard { index_uid: "test-index:0".to_string(), shard_id: 2, - leader_id: "node-2".to_string(), + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 3, + leader_id: "test-ingester-1".to_string(), shard_state: ShardState::Closed as i32, ..Default::default() }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 4, + leader_id: "test-ingester-2".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 5, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, ], - ); - assert!(table.contains_entry("test-index", "test-source")); + remote_round_robin_idx: AtomicUsize::default(), + }; + let shard = table_entry + .next_open_shard_round_robin(&ingester_pool) + .unwrap(); + assert_eq!(shard.shard_id, 2); - let entry = table.find_entry("test-index", "test-source").unwrap(); - assert_eq!(entry.len(), 2); - assert_eq!(entry.shards()[0].shard_id, 0); - assert_eq!(entry.shards()[1].shard_id, 1); - assert_eq!(entry.local_shards.len(), 1); - assert_eq!(entry.remote_shards.len(), 1); + let shard = table_entry + .next_open_shard_round_robin(&ingester_pool) + .unwrap(); + assert_eq!(shard.shard_id, 5); - assert_eq!(entry.next_shard_round_robin().shard_id, 0); - assert_eq!(entry.next_shard_round_robin().shard_id, 0); + let shard = table_entry + .next_open_shard_round_robin(&ingester_pool) + .unwrap(); + assert_eq!(shard.shard_id, 2); } #[test] - fn test_shard_table_with_empty_local_shards() { - let mut table = ShardTable { - self_node_id: "node-0".into(), - table: HashMap::default(), + fn test_shard_table_entry_close_shards() { + let index_uid: IndexUid = IndexUid::new_2("test-index", 0); + let source_id: SourceId = "test-source".into(); + let mut table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: Vec::new(), + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), + remote_round_robin_idx: AtomicUsize::default(), }; - assert!(!table.contains_entry("test-index", "test-source")); + table_entry.close_shards(&index_uid, &[1]); - table.insert_shards( - "test-index", - "test-source", - vec![ + let mut table_entry = ShardTableEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + local_shards: vec![ Shard { index_uid: "test-index:0".to_string(), - shard_id: 0, - leader_id: "node-1".to_string(), + shard_id: 1, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, ..Default::default() }, Shard { index_uid: "test-index:0".to_string(), - shard_id: 1, - leader_id: "node-2".to_string(), + shard_id: 2, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, ..Default::default() }, Shard { index_uid: "test-index:0".to_string(), - shard_id: 2, - leader_id: "node-2".to_string(), - shard_state: ShardState::Closed as i32, + shard_id: 3, + leader_id: "test-ingester-0".to_string(), + shard_state: ShardState::Open as i32, ..Default::default() }, ], - ); - assert!(table.contains_entry("test-index", "test-source")); - - let entry = table.find_entry("test-index", "test-source").unwrap(); - assert_eq!(entry.len(), 2); - assert_eq!(entry.shards()[0].shard_id, 0); - assert_eq!(entry.shards()[1].shard_id, 1); - assert_eq!(entry.local_shards.len(), 0); - assert_eq!(entry.remote_shards.len(), 2); - - assert_eq!(entry.next_shard_round_robin().shard_id, 0); - assert_eq!(entry.next_shard_round_robin().shard_id, 1); - assert_eq!(entry.next_shard_round_robin().shard_id, 0); + local_round_robin_idx: AtomicUsize::default(), + remote_shards: vec![ + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 5, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 6, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 7, + leader_id: "test-ingester-1".to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }, + ], + remote_round_robin_idx: AtomicUsize::default(), + }; + table_entry.close_shards(&index_uid, &[1, 3, 4, 6, 8]); + assert!(table_entry.local_shards[0].is_closed()); + assert!(table_entry.local_shards[1].is_open()); + assert!(table_entry.local_shards[2].is_closed()); + assert!(table_entry.remote_shards[0].is_open()); + assert!(table_entry.remote_shards[1].is_closed()); + assert!(table_entry.remote_shards[2].is_open()); } } diff --git a/quickwit/quickwit-jaeger/src/integration_tests.rs b/quickwit/quickwit-jaeger/src/integration_tests.rs index d9253991b98..67a59b09e72 100644 --- a/quickwit/quickwit-jaeger/src/integration_tests.rs +++ b/quickwit/quickwit-jaeger/src/integration_tests.rs @@ -54,7 +54,7 @@ use quickwit_proto::opentelemetry::proto::trace::v1::span::{Event as OtlpEvent, use quickwit_proto::opentelemetry::proto::trace::v1::{ ResourceSpans, ScopeSpans, Span as OtlpSpan, Status as OtlpStatus, }; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_search::{ start_searcher_service, SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index ae0cfec5e7f..23e4bf43eb3 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -39,7 +39,7 @@ use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::Storage; use serde::Serialize; diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index 53c31cae5e9..c49a9197128 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -34,7 +34,7 @@ use quickwit_proto::metastore::{ MetastoreResult, MetastoreService, MetastoreServiceClient, UpdateSplitsDeleteOpstampRequest, }; use quickwit_proto::search::SearchRequest; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_search::{jobs_to_leaf_requests, IndexMetasForLeafSearch, SearchJob, SearchJobPlacer}; use serde::Serialize; use tantivy::Inventory; diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs index 5e91dd50851..28b284917bd 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs @@ -32,7 +32,7 @@ use quickwit_metastore::{ use quickwit_proto::metastore::{ IndexMetadataRequest, ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::StorageResolver; use serde::Serialize; diff --git a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs index 957bb7a6665..732ed86cd9a 100644 --- a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs +++ b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs @@ -220,7 +220,7 @@ mod tests { use quickwit_proto::metastore::{ EmptyResponse, ListIndexesMetadataResponse, ListSplitsResponse, MetastoreError, }; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use quickwit_storage::MockStorage; use time::OffsetDateTime; diff --git a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs index ad82fc6b218..9d66f943f1a 100644 --- a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs +++ b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs @@ -28,7 +28,7 @@ use quickwit_metastore::{ListIndexesMetadataRequestExt, ListIndexesMetadataRespo use quickwit_proto::metastore::{ ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde::Serialize; use tracing::{debug, error, info}; diff --git a/quickwit/quickwit-janitor/src/retention_policy_execution.rs b/quickwit/quickwit-janitor/src/retention_policy_execution.rs index aee0f776935..68e680fc41a 100644 --- a/quickwit/quickwit-janitor/src/retention_policy_execution.rs +++ b/quickwit/quickwit-janitor/src/retention_policy_execution.rs @@ -26,7 +26,7 @@ use quickwit_metastore::{ use quickwit_proto::metastore::{ ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::{IndexUid, SplitId}; +use quickwit_proto::types::{IndexUid, SplitId}; use time::OffsetDateTime; use tracing::{info, warn}; diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs index 5514807ed22..0d83315aca9 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs @@ -730,7 +730,7 @@ mod tests { use std::collections::BTreeSet; use quickwit_doc_mapper::tag_pruning::TagFilterAst; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use crate::file_backed_metastore::file_backed_index::split_query_predicate; use crate::{ListSplitsQuery, Split, SplitMetadata, SplitState}; diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs index c25a6ebb1b0..f00847ef9e4 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs @@ -20,7 +20,7 @@ use std::collections::HashMap; use itertools::Itertools; -use quickwit_proto::SourceId; +use quickwit_proto::types::SourceId; use serde::{Deserialize, Serialize}; use super::shards::{SerdeShards, Shards}; diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs index baa5460db96..bcc42ddd597 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -48,7 +48,7 @@ use quickwit_proto::metastore::{ PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_storage::Storage; use regex::RegexSet; use time::OffsetDateTime; @@ -654,7 +654,7 @@ impl MetastoreService for FileBackedMetastore { .read_any(&index_id, |index| Ok(index.metadata().clone())) .await?; if let Some(index_uid) = &request.index_uid { - if index_metadata.index_uid.to_string() != *index_uid { + if index_metadata.index_uid != *index_uid { return Err(MetastoreError::NotFound(EntityKind::Index { index_id: index_id.to_string(), })); diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs index 1e4acd49769..9dd810a99da 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs @@ -20,7 +20,7 @@ use std::collections::HashMap; use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde::{self, Deserialize, Serialize}; use crate::checkpoint::IndexCheckpoint; diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index b22ddd653cf..b1eb4a15454 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -41,7 +41,7 @@ use quickwit_proto::metastore::{ ListSplitsRequest, ListSplitsResponse, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, PublishSplitsRequest, StageSplitsRequest, }; -use quickwit_proto::{IndexUid, SplitId}; +use quickwit_proto::types::{IndexUid, SplitId}; use time::OffsetDateTime; use crate::checkpoint::IndexCheckpointDelta; diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index bff32567eef..cbbef53ef37 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -44,7 +44,7 @@ use quickwit_proto::metastore::{ OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use sqlx::migrate::Migrator; use sqlx::postgres::{PgConnectOptions, PgDatabaseError, PgPoolOptions}; use sqlx::{ConnectOptions, Pool, Postgres, Transaction}; @@ -1513,7 +1513,7 @@ metastore_test_suite!(crate::PostgresqlMetastore); mod tests { use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; use quickwit_proto::metastore::MetastoreService; - use quickwit_proto::IndexUid; + use quickwit_proto::types::IndexUid; use time::OffsetDateTime; use super::{build_query_filter, tags_filter_expression_helper, PostgresqlMetastore}; diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs index 85c59f793e1..25454dc9261 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs @@ -21,7 +21,7 @@ use std::convert::TryInto; use std::str::FromStr; use quickwit_proto::metastore::{DeleteQuery, DeleteTask, MetastoreError, MetastoreResult}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use tracing::error; use crate::{IndexMetadata, Split, SplitMetadata, SplitState}; diff --git a/quickwit/quickwit-metastore/src/split_metadata.rs b/quickwit/quickwit-metastore/src/split_metadata.rs index 6fd9b3d4361..e06550ffba0 100644 --- a/quickwit/quickwit-metastore/src/split_metadata.rs +++ b/quickwit/quickwit-metastore/src/split_metadata.rs @@ -25,7 +25,7 @@ use std::str::FromStr; use std::time::Duration; use byte_unit::Byte; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, DurationMilliSeconds}; use time::OffsetDateTime; diff --git a/quickwit/quickwit-metastore/src/split_metadata_version.rs b/quickwit/quickwit-metastore/src/split_metadata_version.rs index 33b103cd83f..fa37fe33383 100644 --- a/quickwit/quickwit-metastore/src/split_metadata_version.rs +++ b/quickwit/quickwit-metastore/src/split_metadata_version.rs @@ -20,7 +20,7 @@ use std::collections::BTreeSet; use std::ops::{Range, RangeInclusive}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde::{Deserialize, Serialize}; use crate::split_metadata::{utc_now_timestamp, SplitMaturity}; diff --git a/quickwit/quickwit-metastore/src/tests.rs b/quickwit/quickwit-metastore/src/tests.rs index 57acd026610..b5825185aa2 100644 --- a/quickwit/quickwit-metastore/src/tests.rs +++ b/quickwit/quickwit-metastore/src/tests.rs @@ -37,7 +37,7 @@ pub mod test_suite { PublishSplitsRequest, ResetSourceCheckpointRequest, SourceType, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, }; - use quickwit_proto::{IndexUid, Position}; + use quickwit_proto::types::{IndexUid, Position}; use quickwit_query::query_ast::qast_json_helper; use time::OffsetDateTime; use tokio::time::sleep; diff --git a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto index 25a877c06cb..ceeb70b559f 100644 --- a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto +++ b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto @@ -65,13 +65,19 @@ service ControlPlaneService { message GetOrCreateOpenShardsRequest { repeated GetOrCreateOpenShardsSubrequest subrequests = 1; - repeated string unavailable_ingesters = 2; + repeated ClosedShards closed_shards = 2; + repeated string unavailable_leaders = 3; } message GetOrCreateOpenShardsSubrequest { string index_id = 1; string source_id = 2; - repeated uint64 closed_shards = 3; +} + +message ClosedShards { + string index_uid = 1; + string source_id = 2; + repeated uint64 shard_ids = 3; } // TODO: Handle partial failures. diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index 367fff025b4..8d6e4ac229c 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -69,7 +69,7 @@ message PersistSuccess { quickwit.ingest.Position replication_position_inclusive = 4; } -enum PersistFailureKind { +enum PersistFailureReason { SHARD_CLOSED = 0; } @@ -77,7 +77,7 @@ message PersistFailure { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - PersistFailureKind failure_kind = 4; + PersistFailureReason failure_reason = 4; } message SynReplicationMessage { diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 89fe91146bf..0937e38861b 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -23,7 +23,6 @@ package quickwit.metastore; import "quickwit/ingest.proto"; - enum SourceType { CLI = 0; FILE = 1; diff --git a/quickwit/quickwit-proto/protos/quickwit/router.proto b/quickwit/quickwit-proto/protos/quickwit/router.proto index 1921ebad44e..be25f6003fb 100644 --- a/quickwit/quickwit-proto/protos/quickwit/router.proto +++ b/quickwit/quickwit-proto/protos/quickwit/router.proto @@ -17,7 +17,6 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . - syntax = "proto3"; package quickwit.ingest.router; @@ -42,8 +41,14 @@ message IngestSubrequest { } message IngestResponseV2 { - repeated IngestSuccess successes = 1; - repeated IngestFailure failures = 2; + repeated IngestSubresponse subresponses = 1; +} + +message IngestSubresponse { + string index_id = 1; + string source_id = 2; + repeated IngestSuccess successes = 3; + repeated IngestFailure failures = 4; } message IngestSuccess { @@ -55,8 +60,6 @@ message IngestSuccess { } message IngestFailure { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - // quickwit.ingest.DocBatchV2 doc_batch = 4; + uint64 shard_id = 1; + // IngestFailureReason reason = 2; } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index 3cf10b798ca..410f1a9ba65 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -4,8 +4,10 @@ pub struct GetOrCreateOpenShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, - #[prost(string, repeated, tag = "2")] - pub unavailable_ingesters: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(message, repeated, tag = "2")] + pub closed_shards: ::prost::alloc::vec::Vec, + #[prost(string, repeated, tag = "3")] + pub unavailable_leaders: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -15,8 +17,17 @@ pub struct GetOrCreateOpenShardsSubrequest { pub index_id: ::prost::alloc::string::String, #[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 ClosedShards { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, #[prost(uint64, repeated, tag = "3")] - pub closed_shards: ::prost::alloc::vec::Vec, + pub shard_ids: ::prost::alloc::vec::Vec, } /// TODO: Handle partial failures. #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] 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 6bedf10f60a..b060d386c82 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -58,8 +58,8 @@ pub struct PersistFailure { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(enumeration = "PersistFailureKind", tag = "4")] - pub failure_kind: i32, + #[prost(enumeration = "PersistFailureReason", tag = "4")] + pub failure_reason: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -265,17 +265,17 @@ pub struct PingResponse {} #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] -pub enum PersistFailureKind { +pub enum PersistFailureReason { ShardClosed = 0, } -impl PersistFailureKind { +impl PersistFailureReason { /// String value of the enum field names used in the ProtoBuf definition. /// /// The values are not transformed in any way and thus are considered stable /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - PersistFailureKind::ShardClosed => "SHARD_CLOSED", + PersistFailureReason::ShardClosed => "SHARD_CLOSED", } } /// Creates an enum from field names used in the ProtoBuf definition. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index 63d43d901d6..c94574223dc 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -23,8 +23,19 @@ pub struct IngestSubrequest { #[derive(Clone, PartialEq, ::prost::Message)] pub struct IngestResponseV2 { #[prost(message, repeated, tag = "1")] + pub subresponses: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IngestSubresponse { + #[prost(string, tag = "1")] + pub index_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "3")] pub successes: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "2")] + #[prost(message, repeated, tag = "4")] pub failures: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -45,12 +56,8 @@ pub struct IngestSuccess { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct IngestFailure { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, - #[prost(string, tag = "2")] - pub source_id: ::prost::alloc::string::String, - /// quickwit.ingest.DocBatchV2 doc_batch = 4; - #[prost(uint64, tag = "3")] + /// IngestFailureReason reason = 2; + #[prost(uint64, tag = "1")] pub shard_id: u64, } /// BEGIN quickwit-codegen diff --git a/quickwit/quickwit-proto/src/error.rs b/quickwit/quickwit-proto/src/error.rs index 3743b85e805..4dd6f2d1104 100644 --- a/quickwit/quickwit-proto/src/error.rs +++ b/quickwit/quickwit-proto/src/error.rs @@ -33,6 +33,7 @@ pub enum ServiceErrorCode { // Used for APIs that are available in Elasticsearch but not available yet in Quickwit. NotSupportedYet, RateLimited, + Timeout, Unavailable, UnsupportedMediaType, } @@ -47,6 +48,7 @@ impl ServiceErrorCode { ServiceErrorCode::NotFound => tonic::Code::NotFound, ServiceErrorCode::NotSupportedYet => tonic::Code::Unimplemented, ServiceErrorCode::RateLimited => tonic::Code::ResourceExhausted, + ServiceErrorCode::Timeout => tonic::Code::DeadlineExceeded, ServiceErrorCode::Unavailable => tonic::Code::Unavailable, ServiceErrorCode::UnsupportedMediaType => tonic::Code::InvalidArgument, } @@ -62,6 +64,7 @@ impl ServiceErrorCode { ServiceErrorCode::RateLimited => http::StatusCode::TOO_MANY_REQUESTS, ServiceErrorCode::Unavailable => http::StatusCode::SERVICE_UNAVAILABLE, ServiceErrorCode::UnsupportedMediaType => http::StatusCode::UNSUPPORTED_MEDIA_TYPE, + ServiceErrorCode::Timeout => http::StatusCode::REQUEST_TIMEOUT, } } } diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index d04b12380d4..ee776f5f3c5 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -59,6 +59,8 @@ pub enum IngestV2Error { source_id: SourceId, shard_id: ShardId, }, + #[error("request timed out")] + Timeout, } impl From for IngestV2Error { @@ -70,10 +72,11 @@ impl From for IngestV2Error { impl From for tonic::Status { fn from(error: IngestV2Error) -> tonic::Status { let code = match &error { - IngestV2Error::Internal(_) => tonic::Code::Internal, IngestV2Error::IngesterUnavailable { .. } => tonic::Code::Unavailable, - IngestV2Error::ShardUnavailable { .. } => tonic::Code::Unavailable, + IngestV2Error::Internal(_) => tonic::Code::Internal, IngestV2Error::ServiceUnavailable { .. } => tonic::Code::Unavailable, + IngestV2Error::ShardUnavailable { .. } => tonic::Code::Unavailable, + IngestV2Error::Timeout { .. } => tonic::Code::DeadlineExceeded, }; let message = error.to_string(); tonic::Status::new(code, message) @@ -93,6 +96,7 @@ impl ServiceError for IngestV2Error { Self::IngesterUnavailable { .. } => ServiceErrorCode::Unavailable, Self::ShardUnavailable { .. } => ServiceErrorCode::Unavailable, Self::ServiceUnavailable { .. } => ServiceErrorCode::Unavailable, + Self::Timeout { .. } => ServiceErrorCode::Timeout, } } } diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index cb8e95483c4..ae43f9683d2 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -41,7 +41,6 @@ pub mod search; pub mod types; pub use error::{ServiceError, ServiceErrorCode}; -pub use types::*; use crate::search::ReportSplitsRequest; @@ -130,7 +129,7 @@ impl TryFrom for search::SearchRequest { type Error = anyhow::Error; fn try_from(delete_query: metastore::DeleteQuery) -> anyhow::Result { - let index_uid: IndexUid = delete_query.index_uid.into(); + let index_uid: types::IndexUid = delete_query.index_uid.into(); Ok(Self { index_id_patterns: vec![index_uid.index_id().to_string()], query_ast: delete_query.query_ast, diff --git a/quickwit/quickwit-proto/src/metastore/events.rs b/quickwit/quickwit-proto/src/metastore/events.rs index 2ebad1ab6a3..eba37334445 100644 --- a/quickwit/quickwit-proto/src/metastore/events.rs +++ b/quickwit/quickwit-proto/src/metastore/events.rs @@ -25,7 +25,7 @@ use super::{ AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, DeleteSourceRequest, SourceType, ToggleSourceRequest, }; -use crate::{IndexUid, SourceId}; +use crate::types::{IndexUid, SourceId}; /// Delete index event. #[derive(Debug, Clone, Eq, PartialEq)] diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index 9f6f74de56d..a732543f23a 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -22,8 +22,8 @@ use std::fmt; use quickwit_common::retry::Retryable; use serde::{Deserialize, Serialize}; -use crate::types::{IndexId, QueueId, SourceId, SplitId}; -use crate::{IndexUid, ServiceError, ServiceErrorCode}; +use crate::types::{IndexId, IndexUid, QueueId, SourceId, SplitId}; +use crate::{ServiceError, ServiceErrorCode}; pub mod events; diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index a7050596934..1a197f9bcd0 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -76,6 +76,11 @@ impl IndexUid { Self::from_parts(index_id, Ulid::new().to_string()) } + /// TODO: Remove when Trinity lands their refactor for #3943. + pub fn new_2(index_id: impl Into, incarnation_id: impl Into) -> Self { + Self(format!("{}:{}", index_id.into(), incarnation_id.into())) + } + pub fn as_str(&self) -> &str { &self.0 } @@ -127,7 +132,13 @@ impl From for IndexUid { impl PartialEq<&str> for IndexUid { fn eq(&self, other: &&str) -> bool { - self.as_str() == *other + self.0 == *other + } +} + +impl PartialEq for IndexUid { + fn eq(&self, other: &String) -> bool { + self.0 == *other } } @@ -170,6 +181,12 @@ impl Borrow for NodeId { } } +impl Borrow for NodeId { + fn borrow(&self) -> &String { + &self.0 + } +} + impl Borrow for NodeId { fn borrow(&self) -> &NodeIdRef { self.deref() @@ -228,6 +245,12 @@ impl PartialEq<&str> for NodeId { } } +impl PartialEq for NodeId { + fn eq(&self, other: &String) -> bool { + self.as_str() == *other + } +} + #[repr(transparent)] #[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] pub struct NodeIdRef(str); @@ -258,12 +281,6 @@ impl AsRef for NodeIdRef { } } -impl Borrow for NodeId { - fn borrow(&self) -> &String { - &self.0 - } -} - impl Borrow for NodeIdRef { fn borrow(&self) -> &str { &self.0 diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 7b17fe122b9..6162b37d758 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -65,7 +65,7 @@ use quickwit_metastore::{ ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, }; use quickwit_proto::search::{PartialHit, SearchRequest, SearchResponse, SplitIdAndFooterOffsets}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_storage::StorageResolver; pub use service::SearcherContext; use tantivy::DocAddress; diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 2f3a8d72a88..34fedd59305 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -43,7 +43,7 @@ use quickwit_proto::search::{ LeafSearchRequest, LeafSearchResponse, ListTermsRequest, ListTermsResponse, PartialHit, SearchRequest, SearchResponse, SnippetRequest, SortField, SplitIdAndFooterOffsets, }; -use quickwit_proto::{IndexUid, SplitId}; +use quickwit_proto::types::{IndexUid, SplitId}; use quickwit_query::query_ast::{ BoolQuery, QueryAst, QueryAstVisitor, RangeQuery, TermQuery, TermSetQuery, }; diff --git a/quickwit/quickwit-search/src/scroll_context.rs b/quickwit/quickwit-search/src/scroll_context.rs index 62ab4295809..c82e4f15210 100644 --- a/quickwit/quickwit-search/src/scroll_context.rs +++ b/quickwit/quickwit-search/src/scroll_context.rs @@ -28,7 +28,7 @@ use base64::prelude::BASE64_STANDARD; use base64::Engine; use quickwit_metastore::SplitMetadata; use quickwit_proto::search::{LeafSearchResponse, PartialHit, SearchRequest}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde::{Deserialize, Serialize}; use tokio::sync::RwLock; use ttl_cache::TtlCache; diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index 919ec78871d..3a18de69db3 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -25,7 +25,7 @@ use quickwit_proto::metastore::{ MetastoreService, MetastoreServiceClient, }; use quickwit_proto::search::SearchRequest; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use quickwit_query::query_ast::{query_ast_from_user_text, QueryAst}; use serde::Deserialize; use warp::{Filter, Rejection}; diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index 0fc611e81ee..3cd7c6f2512 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -38,7 +38,7 @@ use quickwit_proto::metastore::{ ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, ResetSourceCheckpointRequest, ToggleSourceRequest, }; -use quickwit_proto::IndexUid; +use quickwit_proto::types::IndexUid; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use thiserror::Error; diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index a33c8abbadc..b239a5e9a28 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -28,7 +28,7 @@ use quickwit_proto::ingest::router::{ IngestSubrequest, }; use quickwit_proto::ingest::{DocBatchV2, IngestV2Error}; -use quickwit_proto::IndexId; +use quickwit_proto::types::IndexId; use serde::Deserialize; use thiserror::Error; use warp::{Filter, Rejection}; diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index c0a58fcafce..c5e5a356414 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -86,7 +86,7 @@ use quickwit_proto::metastore::{ MetastoreServiceClient, }; use quickwit_proto::search::ReportSplitsRequest; -use quickwit_proto::NodeId; +use quickwit_proto::types::NodeId; use quickwit_search::{ create_search_client_from_channel, start_searcher_service, SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool,